|
@@ -21,6 +21,7 @@ import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
import java.util.Random;
|
|
|
import java.util.concurrent.Callable;
|
|
|
+import java.util.UUID;
|
|
|
|
|
|
import org.junit.Assume;
|
|
|
import org.junit.Ignore;
|
|
@@ -28,6 +29,7 @@ import org.junit.Test;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FSExceptionMessages;
|
|
@@ -37,30 +39,43 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.azure.NativeAzureFileSystem;
|
|
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
|
|
|
|
|
+import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream;
|
|
|
+import org.apache.hadoop.fs.azurebfs.services.TestAbfsInputStream;
|
|
|
+
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.BYTES_RECEIVED;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.GET_RESPONSES;
|
|
|
+import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.ETAG;
|
|
|
|
|
|
/**
|
|
|
* Test random read operation.
|
|
|
*/
|
|
|
public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
AbstractAbfsScaleTest {
|
|
|
+ private static final int BYTE = 1;
|
|
|
+ private static final int THREE_BYTES = 3;
|
|
|
+ private static final int FIVE_BYTES = 5;
|
|
|
+ private static final int TWENTY_BYTES = 20;
|
|
|
+ private static final int THIRTY_BYTES = 30;
|
|
|
private static final int KILOBYTE = 1024;
|
|
|
private static final int MEGABYTE = KILOBYTE * KILOBYTE;
|
|
|
+ private static final int FOUR_MB = 4 * MEGABYTE;
|
|
|
+ private static final int NINE_MB = 9 * MEGABYTE;
|
|
|
private static final long TEST_FILE_SIZE = 8 * MEGABYTE;
|
|
|
private static final int MAX_ELAPSEDTIMEMS = 20;
|
|
|
private static final int SEQUENTIAL_READ_BUFFER_SIZE = 16 * KILOBYTE;
|
|
|
- private static final int CREATE_BUFFER_SIZE = 26 * KILOBYTE;
|
|
|
|
|
|
private static final int SEEK_POSITION_ONE = 2* KILOBYTE;
|
|
|
private static final int SEEK_POSITION_TWO = 5 * KILOBYTE;
|
|
|
private static final int SEEK_POSITION_THREE = 10 * KILOBYTE;
|
|
|
private static final int SEEK_POSITION_FOUR = 4100 * KILOBYTE;
|
|
|
|
|
|
- private static final Path TEST_FILE_PATH = new Path(
|
|
|
- "/TestRandomRead.txt");
|
|
|
+ private static final int ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE = 16 * MEGABYTE;
|
|
|
+ private static final int DISABLED_READAHEAD_DEPTH = 0;
|
|
|
+
|
|
|
+ private static final String TEST_FILE_PREFIX = "/TestRandomRead";
|
|
|
private static final String WASB = "WASB";
|
|
|
private static final String ABFS = "ABFS";
|
|
|
- private static long testFileLength = 0;
|
|
|
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(ITestAzureBlobFileSystemRandomRead.class);
|
|
@@ -71,9 +86,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
|
|
|
@Test
|
|
|
public void testBasicRead() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testBasicRead");
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
byte[] buffer = new byte[3 * MEGABYTE];
|
|
|
|
|
|
// forward seek and read a kilobyte into first kilobyte of bufferV2
|
|
@@ -99,12 +115,14 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
public void testRandomRead() throws Exception {
|
|
|
Assume.assumeFalse("This test does not support namespace enabled account",
|
|
|
this.getFileSystem().getIsNamespaceEnabled());
|
|
|
- assumeHugeFileExists();
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomRead");
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
try (
|
|
|
FSDataInputStream inputStreamV1
|
|
|
- = this.getFileSystem().open(TEST_FILE_PATH);
|
|
|
+ = this.getFileSystem().open(testPath);
|
|
|
FSDataInputStream inputStreamV2
|
|
|
- = this.getWasbFileSystem().open(TEST_FILE_PATH);
|
|
|
+ = this.getWasbFileSystem().open(testPath);
|
|
|
) {
|
|
|
final int bufferSize = 4 * KILOBYTE;
|
|
|
byte[] bufferV1 = new byte[bufferSize];
|
|
@@ -156,8 +174,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
*/
|
|
|
@Test
|
|
|
public void testSeekToNewSource() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testSeekToNewSource");
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
assertFalse(inputStream.seekToNewSource(0));
|
|
|
}
|
|
|
}
|
|
@@ -169,8 +189,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
*/
|
|
|
@Test
|
|
|
public void testSkipBounds() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testSkipBounds");
|
|
|
+ long testFileLength = assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
|
|
|
|
|
long skipped = inputStream.skip(-1);
|
|
@@ -208,8 +230,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
*/
|
|
|
@Test
|
|
|
public void testValidateSeekBounds() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testValidateSeekBounds");
|
|
|
+ long testFileLength = assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
|
|
|
|
|
inputStream.seek(0);
|
|
@@ -257,8 +281,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
*/
|
|
|
@Test
|
|
|
public void testSeekAndAvailableAndPosition() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testSeekAndAvailableAndPosition");
|
|
|
+ long testFileLength = assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
|
|
|
byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
|
|
|
byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
|
|
@@ -321,8 +347,10 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
*/
|
|
|
@Test
|
|
|
public void testSkipAndAvailableAndPosition() throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
- try (FSDataInputStream inputStream = this.getFileSystem().open(TEST_FILE_PATH)) {
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testSkipAndAvailableAndPosition");
|
|
|
+ long testFileLength = assumeHugeFileExists(testPath);
|
|
|
+
|
|
|
+ try (FSDataInputStream inputStream = this.getFileSystem().open(testPath)) {
|
|
|
byte[] expected1 = {(byte) 'a', (byte) 'b', (byte) 'c'};
|
|
|
byte[] expected2 = {(byte) 'd', (byte) 'e', (byte) 'f'};
|
|
|
byte[] expected3 = {(byte) 'b', (byte) 'c', (byte) 'd'};
|
|
@@ -385,15 +413,16 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
@Test
|
|
|
public void testSequentialReadAfterReverseSeekPerformance()
|
|
|
throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testSequentialReadAfterReverseSeekPerformance");
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
final int maxAttempts = 10;
|
|
|
final double maxAcceptableRatio = 1.01;
|
|
|
double beforeSeekElapsedMs = 0, afterSeekElapsedMs = 0;
|
|
|
double ratio = Double.MAX_VALUE;
|
|
|
for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
|
|
|
- beforeSeekElapsedMs = sequentialRead(ABFS,
|
|
|
+ beforeSeekElapsedMs = sequentialRead(ABFS, testPath,
|
|
|
this.getFileSystem(), false);
|
|
|
- afterSeekElapsedMs = sequentialRead(ABFS,
|
|
|
+ afterSeekElapsedMs = sequentialRead(ABFS, testPath,
|
|
|
this.getFileSystem(), true);
|
|
|
ratio = afterSeekElapsedMs / beforeSeekElapsedMs;
|
|
|
LOG.info((String.format(
|
|
@@ -417,8 +446,8 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
public void testRandomReadPerformance() throws Exception {
|
|
|
Assume.assumeFalse("This test does not support namespace enabled account",
|
|
|
this.getFileSystem().getIsNamespaceEnabled());
|
|
|
- createTestFile();
|
|
|
- assumeHugeFileExists();
|
|
|
+ Path testPath = new Path(TEST_FILE_PREFIX + "_testRandomReadPerformance");
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
|
|
|
final AzureBlobFileSystem abFs = this.getFileSystem();
|
|
|
final NativeAzureFileSystem wasbFs = this.getWasbFileSystem();
|
|
@@ -428,8 +457,8 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
double v1ElapsedMs = 0, v2ElapsedMs = 0;
|
|
|
double ratio = Double.MAX_VALUE;
|
|
|
for (int i = 0; i < maxAttempts && ratio >= maxAcceptableRatio; i++) {
|
|
|
- v1ElapsedMs = randomRead(1, wasbFs);
|
|
|
- v2ElapsedMs = randomRead(2, abFs);
|
|
|
+ v1ElapsedMs = randomRead(1, testPath, wasbFs);
|
|
|
+ v2ElapsedMs = randomRead(2, testPath, abFs);
|
|
|
|
|
|
ratio = v2ElapsedMs / v1ElapsedMs;
|
|
|
|
|
@@ -448,15 +477,112 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
ratio < maxAcceptableRatio);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * With this test we should see a full buffer read being triggered in case
|
|
|
+ * alwaysReadBufferSize is on, else only the requested buffer size.
|
|
|
+ * Hence a seek done few bytes away from last read position will trigger
|
|
|
+ * a network read when alwaysReadBufferSize is off, whereas it will return
|
|
|
+ * from the internal buffer when it is on.
|
|
|
+ * Reading a full buffer size is the Gen1 behaviour.
|
|
|
+ * @throws Throwable
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testAlwaysReadBufferSizeConfig() throws Throwable {
|
|
|
+ testAlwaysReadBufferSizeConfig(false);
|
|
|
+ testAlwaysReadBufferSizeConfig(true);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void testAlwaysReadBufferSizeConfig(boolean alwaysReadBufferSizeConfigValue)
|
|
|
+ throws Throwable {
|
|
|
+ final AzureBlobFileSystem currentFs = getFileSystem();
|
|
|
+ Configuration config = new Configuration(this.getRawConfiguration());
|
|
|
+ config.set("fs.azure.readaheadqueue.depth", "0");
|
|
|
+ config.set("fs.azure.read.alwaysReadBufferSize",
|
|
|
+ Boolean.toString(alwaysReadBufferSizeConfigValue));
|
|
|
+
|
|
|
+ final Path testFile = new Path("/FileName_"
|
|
|
+ + UUID.randomUUID().toString());
|
|
|
+
|
|
|
+ final AzureBlobFileSystem fs = createTestFile(testFile, 16 * MEGABYTE,
|
|
|
+ 1 * MEGABYTE, config);
|
|
|
+ String eTag = fs.getAbfsClient()
|
|
|
+ .getPathStatus(testFile.toUri().getPath(), false)
|
|
|
+ .getResult()
|
|
|
+ .getResponseHeader(ETAG);
|
|
|
+
|
|
|
+ TestAbfsInputStream testInputStream = new TestAbfsInputStream();
|
|
|
+
|
|
|
+ AbfsInputStream inputStream = testInputStream.getAbfsInputStream(
|
|
|
+ fs.getAbfsClient(),
|
|
|
+ testFile.getName(), ALWAYS_READ_BUFFER_SIZE_TEST_FILE_SIZE, eTag,
|
|
|
+ DISABLED_READAHEAD_DEPTH, FOUR_MB,
|
|
|
+ alwaysReadBufferSizeConfigValue, FOUR_MB);
|
|
|
+
|
|
|
+ long connectionsAtStart = fs.getInstrumentationMap()
|
|
|
+ .get(GET_RESPONSES.getStatName());
|
|
|
+
|
|
|
+ long dateSizeReadStatAtStart = fs.getInstrumentationMap()
|
|
|
+ .get(BYTES_RECEIVED.getStatName());
|
|
|
+
|
|
|
+ long newReqCount = 0;
|
|
|
+ long newDataSizeRead = 0;
|
|
|
+
|
|
|
+ byte[] buffer20b = new byte[TWENTY_BYTES];
|
|
|
+ byte[] buffer30b = new byte[THIRTY_BYTES];
|
|
|
+ byte[] byteBuffer5 = new byte[FIVE_BYTES];
|
|
|
+
|
|
|
+ // first read
|
|
|
+ // if alwaysReadBufferSize is off, this is a sequential read
|
|
|
+ inputStream.read(byteBuffer5, 0, FIVE_BYTES);
|
|
|
+ newReqCount++;
|
|
|
+ newDataSizeRead += FOUR_MB;
|
|
|
+
|
|
|
+ assertAbfsStatistics(GET_RESPONSES, connectionsAtStart + newReqCount,
|
|
|
+ fs.getInstrumentationMap());
|
|
|
+ assertAbfsStatistics(BYTES_RECEIVED,
|
|
|
+ dateSizeReadStatAtStart + newDataSizeRead, fs.getInstrumentationMap());
|
|
|
+
|
|
|
+ // second read beyond that the buffer holds
|
|
|
+ // if alwaysReadBufferSize is off, this is a random read. Reads only
|
|
|
+ // incoming buffer size
|
|
|
+ // else, reads a buffer size
|
|
|
+ inputStream.seek(NINE_MB);
|
|
|
+ inputStream.read(buffer20b, 0, BYTE);
|
|
|
+ newReqCount++;
|
|
|
+ if (alwaysReadBufferSizeConfigValue) {
|
|
|
+ newDataSizeRead += FOUR_MB;
|
|
|
+ } else {
|
|
|
+ newDataSizeRead += TWENTY_BYTES;
|
|
|
+ }
|
|
|
+
|
|
|
+ assertAbfsStatistics(GET_RESPONSES, connectionsAtStart + newReqCount, fs.getInstrumentationMap());
|
|
|
+ assertAbfsStatistics(BYTES_RECEIVED,
|
|
|
+ dateSizeReadStatAtStart + newDataSizeRead, fs.getInstrumentationMap());
|
|
|
+
|
|
|
+ // third read adjacent to second but not exactly sequential.
|
|
|
+ // if alwaysReadBufferSize is off, this is another random read
|
|
|
+ // else second read would have read this too.
|
|
|
+ inputStream.seek(NINE_MB + TWENTY_BYTES + THREE_BYTES);
|
|
|
+ inputStream.read(buffer30b, 0, THREE_BYTES);
|
|
|
+ if (!alwaysReadBufferSizeConfigValue) {
|
|
|
+ newReqCount++;
|
|
|
+ newDataSizeRead += THIRTY_BYTES;
|
|
|
+ }
|
|
|
+
|
|
|
+ assertAbfsStatistics(GET_RESPONSES, connectionsAtStart + newReqCount, fs.getInstrumentationMap());
|
|
|
+ assertAbfsStatistics(BYTES_RECEIVED, dateSizeReadStatAtStart + newDataSizeRead, fs.getInstrumentationMap());
|
|
|
+ }
|
|
|
|
|
|
private long sequentialRead(String version,
|
|
|
+ Path testPath,
|
|
|
FileSystem fs,
|
|
|
boolean afterReverseSeek) throws IOException {
|
|
|
byte[] buffer = new byte[SEQUENTIAL_READ_BUFFER_SIZE];
|
|
|
long totalBytesRead = 0;
|
|
|
long bytesRead = 0;
|
|
|
|
|
|
- try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
|
|
|
+ long testFileLength = fs.getFileStatus(testPath).getLen();
|
|
|
+ try(FSDataInputStream inputStream = fs.open(testPath)) {
|
|
|
if (afterReverseSeek) {
|
|
|
while (bytesRead > 0 && totalBytesRead < 4 * MEGABYTE) {
|
|
|
bytesRead = inputStream.read(buffer);
|
|
@@ -487,14 +613,14 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private long randomRead(int version, FileSystem fs) throws Exception {
|
|
|
- assumeHugeFileExists();
|
|
|
+ private long randomRead(int version, Path testPath, FileSystem fs) throws Exception {
|
|
|
+ assumeHugeFileExists(testPath);
|
|
|
final long minBytesToRead = 2 * MEGABYTE;
|
|
|
Random random = new Random();
|
|
|
byte[] buffer = new byte[8 * KILOBYTE];
|
|
|
long totalBytesRead = 0;
|
|
|
long bytesRead = 0;
|
|
|
- try(FSDataInputStream inputStream = fs.open(TEST_FILE_PATH)) {
|
|
|
+ try(FSDataInputStream inputStream = fs.open(testPath)) {
|
|
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
|
|
do {
|
|
|
bytesRead = inputStream.read(buffer);
|
|
@@ -526,28 +652,48 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
return bytes / 1000.0 * 8 / milliseconds;
|
|
|
}
|
|
|
|
|
|
- private void createTestFile() throws Exception {
|
|
|
- final AzureBlobFileSystem fs = this.getFileSystem();
|
|
|
- if (fs.exists(TEST_FILE_PATH)) {
|
|
|
- FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
|
|
|
- if (status.getLen() >= TEST_FILE_SIZE) {
|
|
|
- return;
|
|
|
+ private long createTestFile(Path testPath) throws Exception {
|
|
|
+ createTestFile(testPath,
|
|
|
+ TEST_FILE_SIZE,
|
|
|
+ MEGABYTE,
|
|
|
+ null);
|
|
|
+
|
|
|
+ return TEST_FILE_SIZE;
|
|
|
+ }
|
|
|
+
|
|
|
+ private AzureBlobFileSystem createTestFile(Path testFilePath, long testFileSize,
|
|
|
+ int createBufferSize, Configuration config) throws Exception {
|
|
|
+ AzureBlobFileSystem fs;
|
|
|
+
|
|
|
+ if (config == null) {
|
|
|
+ config = this.getRawConfiguration();
|
|
|
+ }
|
|
|
+
|
|
|
+ final AzureBlobFileSystem currentFs = getFileSystem();
|
|
|
+ fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(),
|
|
|
+ config);
|
|
|
+
|
|
|
+ if (fs.exists(testFilePath)) {
|
|
|
+ FileStatus status = fs.getFileStatus(testFilePath);
|
|
|
+ if (status.getLen() == testFileSize) {
|
|
|
+ return fs;
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- byte[] buffer = new byte[CREATE_BUFFER_SIZE];
|
|
|
+ byte[] buffer = new byte[createBufferSize];
|
|
|
char character = 'a';
|
|
|
for (int i = 0; i < buffer.length; i++) {
|
|
|
buffer[i] = (byte) character;
|
|
|
character = (character == 'z') ? 'a' : (char) ((int) character + 1);
|
|
|
}
|
|
|
|
|
|
- LOG.info(String.format("Creating test file %s of size: %d ", TEST_FILE_PATH, TEST_FILE_SIZE));
|
|
|
+ LOG.info(String.format("Creating test file %s of size: %d ", testFilePath, testFileSize));
|
|
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
|
|
|
|
|
- try (FSDataOutputStream outputStream = fs.create(TEST_FILE_PATH)) {
|
|
|
+ try (FSDataOutputStream outputStream = fs.create(testFilePath)) {
|
|
|
+ String bufferContents = new String(buffer);
|
|
|
int bytesWritten = 0;
|
|
|
- while (bytesWritten < TEST_FILE_SIZE) {
|
|
|
+ while (bytesWritten < testFileSize) {
|
|
|
outputStream.write(buffer);
|
|
|
bytesWritten += buffer.length;
|
|
|
}
|
|
@@ -557,18 +703,18 @@ public class ITestAzureBlobFileSystemRandomRead extends
|
|
|
outputStream.close();
|
|
|
closeTimer.end("time to close() output stream");
|
|
|
}
|
|
|
- timer.end("time to write %d KB", TEST_FILE_SIZE / 1024);
|
|
|
- testFileLength = fs.getFileStatus(TEST_FILE_PATH).getLen();
|
|
|
-
|
|
|
+ timer.end("time to write %d KB", testFileSize / 1024);
|
|
|
+ return fs;
|
|
|
}
|
|
|
|
|
|
- private void assumeHugeFileExists() throws Exception{
|
|
|
- createTestFile();
|
|
|
+ private long assumeHugeFileExists(Path testPath) throws Exception{
|
|
|
+ long fileSize = createTestFile(testPath);
|
|
|
FileSystem fs = this.getFileSystem();
|
|
|
- ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", TEST_FILE_PATH);
|
|
|
- FileStatus status = fs.getFileStatus(TEST_FILE_PATH);
|
|
|
- ContractTestUtils.assertIsFile(TEST_FILE_PATH, status);
|
|
|
- assertTrue("File " + TEST_FILE_PATH + " is empty", status.getLen() > 0);
|
|
|
+ ContractTestUtils.assertPathExists(this.getFileSystem(), "huge file not created", testPath);
|
|
|
+ FileStatus status = fs.getFileStatus(testPath);
|
|
|
+ ContractTestUtils.assertIsFile(testPath, status);
|
|
|
+ assertTrue("File " + testPath + " is not of expected size " + fileSize + ":actual=" + status.getLen(), status.getLen() == fileSize);
|
|
|
+ return fileSize;
|
|
|
}
|
|
|
|
|
|
private void verifyConsistentReads(FSDataInputStream inputStreamV1,
|