|
@@ -37,7 +37,6 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
-import org.apache.hadoop.fs.StorageStatistics;
|
|
|
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
|
|
import org.apache.hadoop.fs.s3a.S3AFileSystem;
|
|
|
import org.apache.hadoop.fs.s3a.S3ATestUtils;
|
|
@@ -49,10 +48,11 @@ import org.apache.hadoop.util.Progressable;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.*;
|
|
|
import static org.apache.hadoop.fs.s3a.Constants.*;
|
|
|
import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
|
|
|
+import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING;
|
|
|
+import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupCounterStatistic;
|
|
|
+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.IOStatisticsSupport.retrieveIOStatistics;
|
|
|
-import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.snapshotIOStatistics;
|
|
|
|
|
|
/**
|
|
|
* Scale test which creates a huge file.
|
|
@@ -169,7 +169,8 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
// there's lots of logging here, so that a tail -f on the output log
|
|
|
// can give a view of what is happening.
|
|
|
S3AFileSystem fs = getFileSystem();
|
|
|
- StorageStatistics storageStatistics = fs.getStorageStatistics();
|
|
|
+ IOStatistics iostats = fs.getIOStatistics();
|
|
|
+
|
|
|
String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol();
|
|
|
String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol();
|
|
|
Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
|
|
@@ -205,9 +206,9 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
percentage,
|
|
|
writtenMB,
|
|
|
filesizeMB,
|
|
|
- storageStatistics.getLong(putBytes),
|
|
|
+ iostats.counters().get(putBytes),
|
|
|
gaugeValue(putBytesPending),
|
|
|
- storageStatistics.getLong(putRequests),
|
|
|
+ iostats.counters().get(putRequests),
|
|
|
gaugeValue(putRequestsActive),
|
|
|
elapsedTime,
|
|
|
writtenMB / elapsedTime));
|
|
@@ -227,27 +228,27 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
|
|
|
logFSState();
|
|
|
bandwidth(timer, filesize);
|
|
|
LOG.info("Statistics after stream closed: {}", streamStatistics);
|
|
|
- IOStatistics iostats = snapshotIOStatistics(
|
|
|
- retrieveIOStatistics(getFileSystem()));
|
|
|
+
|
|
|
LOG.info("IOStatistics after upload: {}",
|
|
|
demandStringifyIOStatistics(iostats));
|
|
|
- long putRequestCount = storageStatistics.getLong(putRequests);
|
|
|
- Long putByteCount = storageStatistics.getLong(putBytes);
|
|
|
+ long putRequestCount = lookupCounterStatistic(iostats, putRequests);
|
|
|
+ long putByteCount = lookupCounterStatistic(iostats, putBytes);
|
|
|
Assertions.assertThat(putRequestCount)
|
|
|
.describedAs("Put request count from filesystem stats %s",
|
|
|
iostats)
|
|
|
.isGreaterThan(0);
|
|
|
Assertions.assertThat(putByteCount)
|
|
|
- .describedAs("putByteCount count from filesystem stats %s",
|
|
|
- iostats)
|
|
|
+ .describedAs("%s count from filesystem stats %s",
|
|
|
+ putBytes, iostats)
|
|
|
.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)));
|
|
|
- assertEquals("active put requests in \n" + fs,
|
|
|
- 0, gaugeValue(putRequestsActive));
|
|
|
+ verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0);
|
|
|
+ verifyStatisticGaugeValue(iostats,
|
|
|
+ STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
|
|
|
progress.verifyNoFailures(
|
|
|
"Put file " + fileToCreate + " of size " + filesize);
|
|
|
if (streamStatistics != null) {
|