|
@@ -22,7 +22,6 @@ import java.io.IOException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
-import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
|
import java.util.function.IntFunction;
|
|
@@ -51,17 +50,29 @@ import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
|
import org.apache.hadoop.fs.s3a.Statistic;
|
|
|
import org.apache.hadoop.fs.s3a.statistics.BlockOutputStreamStatistics;
|
|
|
import org.apache.hadoop.fs.statistics.IOStatistics;
|
|
|
+import org.apache.hadoop.util.DurationInfo;
|
|
|
import org.apache.hadoop.util.Progressable;
|
|
|
|
|
|
+import static java.util.Objects.requireNonNull;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_BUFFER_SIZE;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_END;
|
|
|
+import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_SPLIT_START;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
|
+import static org.apache.hadoop.fs.s3a.Statistic.MULTIPART_UPLOAD_COMPLETED;
|
|
|
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING;
|
|
|
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertThatStatisticCounter;
|
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic;
|
|
|
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
|
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticGaugeValue;
|
|
|
-import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
|
|
|
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
|
|
|
+import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
|
|
|
+import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS;
|
|
|
|
|
|
/**
|
|
|
* Scale test which creates a huge file.
|
|
@@ -76,9 +87,10 @@ import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSo
|
|
|
*/
|
|
|
@FixMethodOrder(MethodSorters.NAME_ASCENDING)
|
|
|
public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
+
|
|
|
private static final Logger LOG = LoggerFactory.getLogger(
|
|
|
AbstractSTestS3AHugeFiles.class);
|
|
|
- public static final int DEFAULT_UPLOAD_BLOCKSIZE = 64 * _1KB;
|
|
|
+ public static final int DEFAULT_UPLOAD_BLOCKSIZE = 128 * _1KB;
|
|
|
|
|
|
private Path scaleTestDir;
|
|
|
private Path hugefile;
|
|
@@ -94,6 +106,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
scaleTestDir = new Path(getTestPath(), getTestSuiteName());
|
|
|
hugefile = new Path(scaleTestDir, "hugefile");
|
|
|
hugefileRenamed = new Path(scaleTestDir, "hugefileRenamed");
|
|
|
+ uploadBlockSize = uploadBlockSize();
|
|
|
filesize = getTestPropertyBytes(getConf(), KEY_HUGE_FILESIZE,
|
|
|
DEFAULT_HUGE_FILESIZE);
|
|
|
}
|
|
@@ -117,12 +130,22 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
partitionSize = (int) getTestPropertyBytes(conf,
|
|
|
KEY_HUGE_PARTITION_SIZE,
|
|
|
DEFAULT_HUGE_PARTITION_SIZE);
|
|
|
- assertTrue("Partition size too small: " + partitionSize,
|
|
|
- partitionSize >= MULTIPART_MIN_SIZE);
|
|
|
+ Assertions.assertThat(partitionSize)
|
|
|
+ .describedAs("Partition size set in " + KEY_HUGE_PARTITION_SIZE)
|
|
|
+ .isGreaterThanOrEqualTo(MULTIPART_MIN_SIZE);
|
|
|
+ removeBaseAndBucketOverrides(conf,
|
|
|
+ SOCKET_SEND_BUFFER,
|
|
|
+ SOCKET_RECV_BUFFER,
|
|
|
+ MIN_MULTIPART_THRESHOLD,
|
|
|
+ MULTIPART_SIZE,
|
|
|
+ USER_AGENT_PREFIX,
|
|
|
+ FAST_UPLOAD_BUFFER);
|
|
|
+
|
|
|
conf.setLong(SOCKET_SEND_BUFFER, _1MB);
|
|
|
conf.setLong(SOCKET_RECV_BUFFER, _1MB);
|
|
|
conf.setLong(MIN_MULTIPART_THRESHOLD, partitionSize);
|
|
|
conf.setInt(MULTIPART_SIZE, partitionSize);
|
|
|
+ conf.setInt(AWS_S3_VECTOR_ACTIVE_RANGE_READS, 32);
|
|
|
conf.set(USER_AGENT_PREFIX, "STestS3AHugeFileCreate");
|
|
|
conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName());
|
|
|
S3ATestUtils.disableFilesystemCaching(conf);
|
|
@@ -180,6 +203,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
IOStatistics iostats = fs.getIOStatistics();
|
|
|
|
|
|
String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol();
|
|
|
+ String multipartBlockUploads = Statistic.MULTIPART_UPLOAD_PART_PUT.getSymbol();
|
|
|
String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol();
|
|
|
Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
|
|
|
Statistic putBytesPending = Statistic.OBJECT_PUT_BYTES_PENDING;
|
|
@@ -192,13 +216,8 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
true,
|
|
|
uploadBlockSize,
|
|
|
progress)) {
|
|
|
- try {
|
|
|
- streamStatistics = getOutputStreamStatistics(out);
|
|
|
- } catch (ClassCastException e) {
|
|
|
- LOG.info("Wrapped output stream is not block stream: {}",
|
|
|
- out.getWrappedStream());
|
|
|
- streamStatistics = null;
|
|
|
- }
|
|
|
+ streamStatistics = requireNonNull(getOutputStreamStatistics(out),
|
|
|
+ () -> "No iostatistics in " + out);
|
|
|
|
|
|
for (long block = 1; block <= blocks; block++) {
|
|
|
out.write(data);
|
|
@@ -222,6 +241,13 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
writtenMB / elapsedTime));
|
|
|
}
|
|
|
}
|
|
|
+ if (!expectMultipartUpload()) {
|
|
|
+ // it is required that no data has uploaded at this point on a
|
|
|
+ // non-multipart upload
|
|
|
+ Assertions.assertThat(progress.getUploadEvents())
|
|
|
+ .describedAs("upload events in %s", progress)
|
|
|
+ .isEqualTo(0);
|
|
|
+ }
|
|
|
// now close the file
|
|
|
LOG.info("Closing stream {}", out);
|
|
|
LOG.info("Statistics : {}", streamStatistics);
|
|
@@ -235,34 +261,51 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
filesizeMB, uploadBlockSize);
|
|
|
logFSState();
|
|
|
bandwidth(timer, filesize);
|
|
|
- LOG.info("Statistics after stream closed: {}", streamStatistics);
|
|
|
|
|
|
- LOG.info("IOStatistics after upload: {}",
|
|
|
- demandStringifyIOStatistics(iostats));
|
|
|
- long putRequestCount = lookupCounterStatistic(iostats, putRequests);
|
|
|
+ final IOStatistics streamIOstats = streamStatistics.getIOStatistics();
|
|
|
+ LOG.info("Stream IOStatistics after stream closed: {}",
|
|
|
+ ioStatisticsToPrettyString(streamIOstats));
|
|
|
+
|
|
|
+ LOG.info("FileSystem IOStatistics after upload: {}",
|
|
|
+ ioStatisticsToPrettyString(iostats));
|
|
|
+ final String requestKey;
|
|
|
long putByteCount = lookupCounterStatistic(iostats, putBytes);
|
|
|
- Assertions.assertThat(putRequestCount)
|
|
|
- .describedAs("Put request count from filesystem stats %s",
|
|
|
- iostats)
|
|
|
- .isGreaterThan(0);
|
|
|
+ long putRequestCount;
|
|
|
+
|
|
|
+ if (expectMultipartUpload()) {
|
|
|
+ requestKey = multipartBlockUploads;
|
|
|
+ putRequestCount = lookupCounterStatistic(streamIOstats, requestKey);
|
|
|
+ assertThatStatisticCounter(streamIOstats, multipartBlockUploads)
|
|
|
+ .isGreaterThanOrEqualTo(1);
|
|
|
+ verifyStatisticCounterValue(streamIOstats, STREAM_WRITE_BLOCK_UPLOADS, putRequestCount);
|
|
|
+ // non-magic uploads will have completed
|
|
|
+ verifyStatisticCounterValue(streamIOstats, MULTIPART_UPLOAD_COMPLETED.getSymbol(),
|
|
|
+ expectImmediateFileVisibility() ? 1 : 0);
|
|
|
+ } else {
|
|
|
+ // single put
|
|
|
+ requestKey = putRequests;
|
|
|
+ putRequestCount = lookupCounterStatistic(streamIOstats, requestKey);
|
|
|
+ verifyStatisticCounterValue(streamIOstats, putRequests, 1);
|
|
|
+ verifyStatisticCounterValue(streamIOstats, STREAM_WRITE_BLOCK_UPLOADS, 1);
|
|
|
+ verifyStatisticCounterValue(streamIOstats, MULTIPART_UPLOAD_COMPLETED.getSymbol(), 0);
|
|
|
+ }
|
|
|
Assertions.assertThat(putByteCount)
|
|
|
- .describedAs("%s count from filesystem stats %s",
|
|
|
- putBytes, iostats)
|
|
|
+ .describedAs("%s count from stream stats %s",
|
|
|
+ putBytes, streamStatistics)
|
|
|
.isGreaterThan(0);
|
|
|
+
|
|
|
LOG.info("PUT {} bytes in {} operations; {} MB/operation",
|
|
|
putByteCount, putRequestCount,
|
|
|
putByteCount / (putRequestCount * _1MB));
|
|
|
LOG.info("Time per PUT {} nS",
|
|
|
toHuman(timer.nanosPerOperation(putRequestCount)));
|
|
|
verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0);
|
|
|
- verifyStatisticGaugeValue(iostats,
|
|
|
- STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
|
|
|
+ verifyStatisticGaugeValue(iostats, STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
|
|
|
+
|
|
|
progress.verifyNoFailures(
|
|
|
"Put file " + fileToCreate + " of size " + filesize);
|
|
|
- if (streamStatistics != null) {
|
|
|
- assertEquals("actively allocated blocks in " + streamStatistics,
|
|
|
- 0, streamStatistics.getBlocksActivelyAllocated());
|
|
|
- }
|
|
|
+ assertEquals("actively allocated blocks in " + streamStatistics,
|
|
|
+ 0, streamStatistics.getBlocksActivelyAllocated());
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -290,10 +333,45 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
return hugefileRenamed;
|
|
|
}
|
|
|
|
|
|
- protected int getUploadBlockSize() {
|
|
|
+ public int getUploadBlockSize() {
|
|
|
return uploadBlockSize;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the desired upload block size for this test run.
|
|
|
+ * @return the block size
|
|
|
+ */
|
|
|
+ protected int uploadBlockSize() {
|
|
|
+ return DEFAULT_UPLOAD_BLOCKSIZE;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the size of the file.
|
|
|
+ * @return file size
|
|
|
+ */
|
|
|
+ public long getFilesize() {
|
|
|
+ return filesize;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Is this expected to be a multipart upload?
|
|
|
+ * Assertions will change if not.
|
|
|
+ * @return true by default.
|
|
|
+ */
|
|
|
+ protected boolean expectMultipartUpload() {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Is this expected to be a normal file creation with
|
|
|
+ * the output immediately visible?
|
|
|
+ * Assertions will change if not.
|
|
|
+ * @return true by default.
|
|
|
+ */
|
|
|
+ protected boolean expectImmediateFileVisibility() {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
protected int getPartitionSize() {
|
|
|
return partitionSize;
|
|
|
}
|
|
@@ -304,6 +382,7 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
private final class ProgressCallback implements Progressable,
|
|
|
ProgressListener {
|
|
|
private AtomicLong bytesTransferred = new AtomicLong(0);
|
|
|
+ private AtomicLong uploadEvents = new AtomicLong(0);
|
|
|
private AtomicInteger failures = new AtomicInteger(0);
|
|
|
private final ContractTestUtils.NanoTimer timer;
|
|
|
|
|
@@ -339,10 +418,11 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
progressEvent,
|
|
|
writtenMB, elapsedTimeS, writtenMB / elapsedTimeS));
|
|
|
break;
|
|
|
+ case REQUEST_BYTE_TRANSFER_EVENT:
|
|
|
+ uploadEvents.incrementAndGet();
|
|
|
+ break;
|
|
|
default:
|
|
|
- if (eventType.isByteCountEvent()) {
|
|
|
- LOG.debug("Event {}", progressEvent);
|
|
|
- } else {
|
|
|
+ if (!eventType.isByteCountEvent()) {
|
|
|
LOG.info("Event {}", progressEvent);
|
|
|
}
|
|
|
break;
|
|
@@ -352,12 +432,29 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
@Override
|
|
|
public String toString() {
|
|
|
String sb = "ProgressCallback{"
|
|
|
- + "bytesTransferred=" + bytesTransferred +
|
|
|
- ", failures=" + failures +
|
|
|
+ + "bytesTransferred=" + bytesTransferred.get() +
|
|
|
+ ", uploadEvents=" + uploadEvents.get() +
|
|
|
+ ", failures=" + failures.get() +
|
|
|
'}';
|
|
|
return sb;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Get the number of bytes transferred.
|
|
|
+ * @return byte count
|
|
|
+ */
|
|
|
+ private long getBytesTransferred() {
|
|
|
+ return bytesTransferred.get();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the number of event callbacks.
|
|
|
+ * @return count of byte transferred events.
|
|
|
+ */
|
|
|
+ private long getUploadEvents() {
|
|
|
+ return uploadEvents.get();
|
|
|
+ }
|
|
|
+
|
|
|
private void verifyNoFailures(String operation) {
|
|
|
assertEquals("Failures in " + operation + ": " + this, 0, failures.get());
|
|
|
}
|
|
@@ -467,15 +564,42 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
rangeList.add(FileRange.createFileRange(2820861, 156770));
|
|
|
IntFunction<ByteBuffer> allocate = ByteBuffer::allocate;
|
|
|
FileSystem fs = getFileSystem();
|
|
|
- CompletableFuture<FSDataInputStream> builder =
|
|
|
- fs.openFile(hugefile).build();
|
|
|
- try (FSDataInputStream in = builder.get()) {
|
|
|
- in.readVectored(rangeList, allocate);
|
|
|
- byte[] readFullRes = new byte[(int)filesize];
|
|
|
+
|
|
|
+ // read into a buffer first
|
|
|
+ // using sequential IO
|
|
|
+
|
|
|
+ int validateSize = (int) Math.min(filesize, 10 * _1MB);
|
|
|
+ byte[] readFullRes;
|
|
|
+ IOStatistics sequentialIOStats, vectorIOStats;
|
|
|
+ try (FSDataInputStream in = fs.openFile(hugefile)
|
|
|
+ .opt(FS_OPTION_OPENFILE_LENGTH, validateSize) // lets us actually force a shorter read
|
|
|
+ .opt(FS_OPTION_OPENFILE_SPLIT_START, 0)
|
|
|
+ .opt(FS_OPTION_OPENFILE_SPLIT_END, validateSize)
|
|
|
+ .opt(FS_OPTION_OPENFILE_READ_POLICY, "sequential")
|
|
|
+ .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, uploadBlockSize)
|
|
|
+ .build().get();
|
|
|
+ DurationInfo ignored = new DurationInfo(LOG, "Sequential read of %,d bytes",
|
|
|
+ validateSize)) {
|
|
|
+ readFullRes = new byte[validateSize];
|
|
|
in.readFully(0, readFullRes);
|
|
|
+ sequentialIOStats = in.getIOStatistics();
|
|
|
+ }
|
|
|
+
|
|
|
+ // now do a vector IO read
|
|
|
+ try (FSDataInputStream in = fs.openFile(hugefile)
|
|
|
+ .opt(FS_OPTION_OPENFILE_LENGTH, filesize)
|
|
|
+ .opt(FS_OPTION_OPENFILE_READ_POLICY, "vector, random")
|
|
|
+ .build().get();
|
|
|
+ DurationInfo ignored = new DurationInfo(LOG, "Vector Read")) {
|
|
|
+
|
|
|
+ in.readVectored(rangeList, allocate);
|
|
|
// Comparing vectored read results with read fully.
|
|
|
validateVectoredReadResult(rangeList, readFullRes);
|
|
|
+ vectorIOStats = in.getIOStatistics();
|
|
|
}
|
|
|
+
|
|
|
+ LOG.info("Bulk read IOStatistics={}", ioStatisticsToPrettyString(sequentialIOStats));
|
|
|
+ LOG.info("Vector IOStatistics={}", ioStatisticsToPrettyString(vectorIOStats));
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -493,7 +617,12 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
byte[] data = new byte[uploadBlockSize];
|
|
|
|
|
|
ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer();
|
|
|
- try (FSDataInputStream in = fs.open(hugefile, uploadBlockSize)) {
|
|
|
+ try (FSDataInputStream in = fs.openFile(hugefile)
|
|
|
+ .withFileStatus(status)
|
|
|
+ .opt(FS_OPTION_OPENFILE_BUFFER_SIZE, uploadBlockSize)
|
|
|
+ .opt(FS_OPTION_OPENFILE_READ_POLICY, FS_OPTION_OPENFILE_READ_POLICY_WHOLE_FILE)
|
|
|
+ .build().get();
|
|
|
+ DurationInfo ignored = new DurationInfo(LOG, "Vector Read")) {
|
|
|
for (long block = 0; block < blocks; block++) {
|
|
|
in.readFully(data);
|
|
|
}
|