|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.contract.localfs;
|
|
|
|
|
|
+import java.io.EOFException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileRange;
|
|
|
+import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest;
|
|
@@ -52,9 +54,33 @@ public class TestLocalFSContractVectoredRead extends AbstractContractVectoredRea
|
|
|
|
|
|
@Test
|
|
|
public void testChecksumValidationDuringVectoredRead() throws Exception {
|
|
|
- Path testPath = path("big_range_checksum");
|
|
|
+ Path testPath = path("big_range_checksum_file");
|
|
|
+ List<FileRange> someRandomRanges = new ArrayList<>();
|
|
|
+ someRandomRanges.add(FileRange.createFileRange(10, 1024));
|
|
|
+ someRandomRanges.add(FileRange.createFileRange(1025, 1024));
|
|
|
+ validateCheckReadException(testPath, DATASET_LEN, someRandomRanges);
|
|
|
+ }
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Test for file size less than checksum chunk size.
|
|
|
+ * {@code ChecksumFileSystem#bytesPerChecksum}.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testChecksumValidationDuringVectoredReadSmallFile() throws Exception {
|
|
|
+ Path testPath = path("big_range_checksum_file");
|
|
|
+ final int length = 471;
|
|
|
+ List<FileRange> smallFileRanges = new ArrayList<>();
|
|
|
+ smallFileRanges.add(FileRange.createFileRange(10, 50));
|
|
|
+ smallFileRanges.add(FileRange.createFileRange(100, 20));
|
|
|
+ validateCheckReadException(testPath, length, smallFileRanges);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void validateCheckReadException(Path testPath,
|
|
|
+ int length,
|
|
|
+ List<FileRange> ranges) throws Exception {
|
|
|
LocalFileSystem localFs = (LocalFileSystem) getFileSystem();
|
|
|
- final byte[] datasetCorrect = ContractTestUtils.dataset(DATASET_LEN, 'a', 32);
|
|
|
+ final byte[] datasetCorrect = ContractTestUtils.dataset(length, 'a', 32);
|
|
|
try (FSDataOutputStream out = localFs.create(testPath, true)){
|
|
|
out.write(datasetCorrect);
|
|
|
}
|
|
@@ -63,24 +89,55 @@ public class TestLocalFSContractVectoredRead extends AbstractContractVectoredRea
|
|
|
.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 at {} ", checksumPath)
|
|
|
+ .isTrue();
|
|
|
+ CompletableFuture<FSDataInputStream> fis = localFs.openFile(testPath).build();
|
|
|
+ List<FileRange> smallRange = new ArrayList<>();
|
|
|
+ smallRange.add(FileRange.createFileRange(1000, 71));
|
|
|
+ try (FSDataInputStream in = fis.get()){
|
|
|
+ in.readVectored(smallRange, getAllocate());
|
|
|
+ validateVectoredReadResult(smallRange, datasetCorrect);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Overriding in checksum fs as vectored read api fails fast
|
|
|
+ * in case of EOF requested range.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public void testEOFRanges() throws Exception {
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
+ List<FileRange> fileRanges = new ArrayList<>();
|
|
|
+ fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100));
|
|
|
+ verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class);
|
|
|
+ }
|
|
|
}
|