[
https://issues.apache.org/jira/browse/HADOOP-18439?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17601481#comment-17601481
]
ASF GitHub Bot commented on HADOOP-18439:
-----------------------------------------
steveloughran commented on code in PR #4862:
URL: https://github.com/apache/hadoop/pull/4862#discussion_r965178779
##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java:
##########
@@ -63,24 +87,55 @@ public void testChecksumValidationDuringVectoredRead()
throws Exception {
.describedAs("Checksum file should be present")
.isTrue();
CompletableFuture<FSDataInputStream> fis =
localFs.openFile(testPath).build();
- List<FileRange> someRandomRanges = new ArrayList<>();
- someRandomRanges.add(FileRange.createFileRange(10, 1024));
- someRandomRanges.add(FileRange.createFileRange(1025, 1024));
try (FSDataInputStream in = fis.get()){
- in.readVectored(someRandomRanges, getAllocate());
- validateVectoredReadResult(someRandomRanges, datasetCorrect);
+ in.readVectored(ranges, getAllocate());
+ validateVectoredReadResult(ranges, datasetCorrect);
}
- final byte[] datasetCorrupted = ContractTestUtils.dataset(DATASET_LEN,
'a', 64);
+ final byte[] datasetCorrupted = ContractTestUtils.dataset(length, 'a', 64);
try (FSDataOutputStream out = localFs.getRaw().create(testPath, true)){
out.write(datasetCorrupted);
}
CompletableFuture<FSDataInputStream> fisN =
localFs.openFile(testPath).build();
try (FSDataInputStream in = fisN.get()){
- in.readVectored(someRandomRanges, getAllocate());
+ in.readVectored(ranges, getAllocate());
// Expect checksum exception when data is updated directly through
// raw local fs instance.
intercept(ChecksumException.class,
- () -> validateVectoredReadResult(someRandomRanges,
datasetCorrupted));
+ () -> validateVectoredReadResult(ranges, datasetCorrupted));
+ }
+ }
+ @Test
+ public void tesChecksumVectoredReadBoundaries() throws Exception {
+ Path testPath = path("boundary_range_checksum_file");
+ final int length = 1071;
+ LocalFileSystem localFs = (LocalFileSystem) getFileSystem();
+ final byte[] datasetCorrect = ContractTestUtils.dataset(length, 'a', 32);
+ try (FSDataOutputStream out = localFs.create(testPath, true)){
+ out.write(datasetCorrect);
+ }
+ Path checksumPath = localFs.getChecksumFile(testPath);
+ Assertions.assertThat(localFs.exists(checksumPath))
+ .describedAs("Checksum file should be present")
Review Comment:
add path in message
> Fix VectoredIO for LocalFileSystem when checksum is enabled.
> ------------------------------------------------------------
>
> Key: HADOOP-18439
> URL: https://issues.apache.org/jira/browse/HADOOP-18439
> Project: Hadoop Common
> Issue Type: Sub-task
> Components: common
> Affects Versions: 3.3.9
> Reporter: Mukund Thakur
> Assignee: Mukund Thakur
> Priority: Major
> Labels: pull-request-available
>
> While merging the ranges in CheckSumFs, they are rounded up based on the
> value of checksum bytes size
> which leads to some ranges crossing the EOF thus they need to be fixed else
> it will cause EOFException during actual reads.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]