瀏覽代碼

Revert "HADOOP-17451. IOStatistics test failures in S3A code. (#2594)"

This reverts commit d3014e01f3538c6b161b48fa297ba8afeb002b30.
(fixing commit text before it is frozen)
Steve Loughran 4 年之前
父節點
當前提交
05c9c2ed02

+ 5 - 28
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/StorageStatisticsFromIOStatistics.java

@@ -67,46 +67,23 @@ public class StorageStatisticsFromIOStatistics
   public Iterator<LongStatistic> getLongStatistics() {
     final Set<Map.Entry<String, Long>> counters = counters()
         .entrySet();
-    final Set<LongStatistic> statisticSet = counters.stream().map(
-        this::toLongStatistic)
-        .collect(Collectors.toSet());
-
-    // add the gauges
-    gauges().entrySet().forEach(entry ->
-        statisticSet.add(toLongStatistic(entry)));
-    return statisticSet.iterator();
-  }
-
-  /**
-   * Convert a counter/gauge entry to a long statistics.
-   * @param e entry
-   * @return statistic
-   */
-  private LongStatistic toLongStatistic(final Map.Entry<String, Long> e) {
-    return new LongStatistic(e.getKey(), e.getValue());
+    return counters.stream().map(e ->
+        new StorageStatistics.LongStatistic(e.getKey(), e.getValue()))
+        .collect(Collectors.toSet()).iterator();
   }
 
   private Map<String, Long> counters() {
     return ioStatistics.counters();
   }
 
-  private Map<String, Long> gauges() {
-    return ioStatistics.gauges();
-  }
-
   @Override
   public Long getLong(final String key) {
-    Long l = counters().get(key);
-    if (l == null) {
-      l = gauges().get(key);
-    }
-    return l;
+    return counters().get(key);
   }
 
   @Override
   public boolean isTracked(final String key) {
-    return counters().containsKey(key)
-        || gauges().containsKey(key);
+    return counters().containsKey(key);
   }
 
   @Override

+ 26 - 12
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -64,8 +64,10 @@ import org.apache.hadoop.metrics2.lib.MutableQuantiles;
 import java.io.Closeable;
 import java.net.URI;
 import java.time.Duration;
+import java.util.Arrays;
 import java.util.EnumSet;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -180,6 +182,20 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
    */
   private final IOStatisticsStore instanceIOStatistics;
 
+  /**
+   * Gauges to create.
+   * <p></p>
+   * All statistics which are not gauges or quantiles
+   * are registered as counters.
+   */
+  private static final Statistic[] GAUGES_TO_CREATE = {
+      OBJECT_PUT_REQUESTS_ACTIVE,
+      OBJECT_PUT_BYTES_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_ACTIVE,
+      STREAM_WRITE_BLOCK_UPLOADS_PENDING,
+      STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
+  };
+
   /**
    * Construct the instrumentation for a filesystem.
    * @param name URI of filesystem.
@@ -195,6 +211,10 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
     // create the builder
     IOStatisticsStoreBuilder storeBuilder = iostatisticsStore();
 
+    // add the gauges
+    List<Statistic> gauges = Arrays.asList(GAUGES_TO_CREATE);
+    gauges.forEach(this::gauge);
+
     // declare all counter statistics
     EnumSet.allOf(Statistic.class).stream()
         .filter(statistic ->
@@ -203,14 +223,6 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
           counter(stat);
           storeBuilder.withCounters(stat.getSymbol());
         });
-    // declare all gauge statistics
-    EnumSet.allOf(Statistic.class).stream()
-        .filter(statistic ->
-            statistic.getType() == StatisticTypeEnum.TYPE_GAUGE)
-        .forEach(stat -> {
-          gauge(stat);
-          storeBuilder.withGauges(stat.getSymbol());
-        });
 
     // and durations
     EnumSet.allOf(Statistic.class).stream()
@@ -1340,13 +1352,15 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
       this.filesystemStatistics = filesystemStatistics;
       IOStatisticsStore st = iostatisticsStore()
           .withCounters(
-              STREAM_WRITE_BLOCK_UPLOADS.getSymbol(),
+              StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS,
               STREAM_WRITE_BYTES.getSymbol(),
               STREAM_WRITE_EXCEPTIONS.getSymbol(),
-              STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol(),
+              StreamStatisticNames.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING,
+              STREAM_WRITE_TOTAL_TIME.getSymbol(),
               STREAM_WRITE_QUEUE_DURATION.getSymbol(),
               STREAM_WRITE_TOTAL_DATA.getSymbol(),
-              STREAM_WRITE_TOTAL_TIME.getSymbol())
+              STREAM_WRITE_EXCEPTIONS.getSymbol(),
+              STREAM_WRITE_EXCEPTIONS_COMPLETING_UPLOADS.getSymbol())
           .withGauges(
               STREAM_WRITE_BLOCK_UPLOADS_PENDING.getSymbol(),
               STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol())
@@ -1456,7 +1470,7 @@ public class S3AInstrumentation implements Closeable, MetricsSource,
     @Override
     public void bytesTransferred(long byteCount) {
       bytesUploaded.addAndGet(byteCount);
-      incAllGauges(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount);
+      incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING, -byteCount);
     }
 
     @Override

+ 2 - 8
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/ITestPartialRenamesDeletes.java

@@ -76,7 +76,6 @@ import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.removeUndel
 import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.toPathList;
 import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.assertFileCount;
 import static org.apache.hadoop.fs.s3a.test.ExtraAssertions.extractCause;
-import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsSourceToString;
 import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
 import static org.apache.hadoop.test.LambdaTestUtils.eval;
 
@@ -684,8 +683,7 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
           readOnlyFiles.size());
       rejectionCount.assertDiffEquals("Wrong rejection count",
           readOnlyFiles.size());
-      reset(rejectionCount, deleteVerbCount, deleteObjectCount,
-          bulkDeleteVerbCount);
+      reset(rejectionCount, deleteVerbCount, deleteObjectCount);
     }
     // all the files are still there? (avoid in scale test due to cost)
     if (!scaleTest) {
@@ -694,13 +692,9 @@ public class ITestPartialRenamesDeletes extends AbstractS3ATestBase {
 
     describe("Trying to delete upper-level directory");
     ex = expectDeleteForbidden(basePath);
-    String iostats = ioStatisticsSourceToString(roleFS);
-
     if (multiDelete) {
       // multi-delete status checks
-      deleteVerbCount.assertDiffEquals("Wrong delete request count", 0);
-      bulkDeleteVerbCount.assertDiffEquals(
-          "Wrong count of delete operations in " + iostats, 1);
+      deleteVerbCount.assertDiffEquals("Wrong delete count", 1);
       MultiObjectDeleteException mde = extractCause(
           MultiObjectDeleteException.class, ex);
       List<MultiObjectDeleteSupport.KeyPath> undeletedKeyPaths =

+ 2 - 3
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java

@@ -475,7 +475,7 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
 
   /**
    * Execute a closure expecting a specific number of HEAD/LIST calls
-   * on <i>raw</i> S3 stores only. The operation is always evaluated.
+   * on <i>raw</i> S3 stores only.
    * @param cost expected cost
    * @param eval closure to evaluate
    * @param <T> return type of closure
@@ -484,8 +484,7 @@ public class AbstractS3ACostTest extends AbstractS3ATestBase {
   protected <T> T verifyRaw(
       OperationCost cost,
       Callable<T> eval) throws Exception {
-    return verifyMetrics(eval,
-        whenRaw(cost), OperationCostValidator.always());
+    return verifyMetrics(eval, whenRaw(cost));
   }
 
   /**

+ 5 - 9
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java

@@ -121,21 +121,17 @@ public class ITestS3ADeleteCost extends AbstractS3ACostTest {
         with(DIRECTORIES_DELETED, 0),
         with(FILES_DELETED, 1),
 
-        // a single DELETE call is made to delete the object
-        with(OBJECT_DELETE_REQUEST, DELETE_OBJECT_REQUEST),
-
         // keeping: create no parent dirs or delete parents
         withWhenKeeping(DIRECTORIES_CREATED, 0),
-        withWhenKeeping(OBJECT_BULK_DELETE_REQUEST, 0),
+        withWhenKeeping(OBJECT_DELETE_OBJECTS, DELETE_OBJECT_REQUEST),
 
         // deleting: create a parent and delete any of its parents
         withWhenDeleting(DIRECTORIES_CREATED, 1),
-        // a bulk delete for all parents is issued.
-        // the number of objects in it depends on the depth of the tree;
-        // don't worry about that
-        withWhenDeleting(OBJECT_BULK_DELETE_REQUEST, DELETE_MARKER_REQUEST)
+        // two objects will be deleted
+        withWhenDeleting(OBJECT_DELETE_OBJECTS,
+            DELETE_OBJECT_REQUEST
+                + DELETE_MARKER_REQUEST)
     );
-
     // there is an empty dir for a parent
     S3AFileStatus status = verifyRawInnerGetFileStatus(dir, true,
         StatusProbeEnum.ALL, GET_FILE_STATUS_ON_DIR);

+ 14 - 15
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java

@@ -37,6 +37,7 @@ 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;
@@ -48,11 +49,10 @@ 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,8 +169,7 @@ 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();
-    IOStatistics iostats = fs.getIOStatistics();
-
+    StorageStatistics storageStatistics = fs.getStorageStatistics();
     String putRequests = Statistic.OBJECT_PUT_REQUESTS.getSymbol();
     String putBytes = Statistic.OBJECT_PUT_BYTES.getSymbol();
     Statistic putRequestsActive = Statistic.OBJECT_PUT_REQUESTS_ACTIVE;
@@ -206,9 +205,9 @@ public abstract class AbstractSTestS3AHugeFiles extends S3AScaleTestBase {
               percentage,
               writtenMB,
               filesizeMB,
-              iostats.counters().get(putBytes),
+              storageStatistics.getLong(putBytes),
               gaugeValue(putBytesPending),
-              iostats.counters().get(putRequests),
+              storageStatistics.getLong(putRequests),
               gaugeValue(putRequestsActive),
               elapsedTime,
               writtenMB / elapsedTime));
@@ -228,27 +227,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 = lookupCounterStatistic(iostats, putRequests);
-    long putByteCount = lookupCounterStatistic(iostats, putBytes);
+    long putRequestCount = storageStatistics.getLong(putRequests);
+    Long putByteCount = storageStatistics.getLong(putBytes);
     Assertions.assertThat(putRequestCount)
         .describedAs("Put request count from filesystem stats %s",
             iostats)
         .isGreaterThan(0);
     Assertions.assertThat(putByteCount)
-        .describedAs("%s count from filesystem stats %s",
-            putBytes, iostats)
+        .describedAs("putByteCount count from filesystem stats %s",
+            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)));
-    verifyStatisticGaugeValue(iostats, putRequestsActive.getSymbol(), 0);
-    verifyStatisticGaugeValue(iostats,
-        STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol(), 0);
+    assertEquals("active put requests in \n" + fs,
+        0, gaugeValue(putRequestsActive));
     progress.verifyNoFailures(
         "Put file " + fileToCreate + " of size " + filesize);
     if (streamStatistics != null) {

+ 8 - 5
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/S3AScaleTestBase.java

@@ -23,9 +23,11 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.s3a.AbstractS3ATestBase;
 import org.apache.hadoop.fs.s3a.S3AInputStream;
+import org.apache.hadoop.fs.s3a.S3AInstrumentation;
 import org.apache.hadoop.fs.s3a.S3ATestConstants;
 import org.apache.hadoop.fs.s3a.Statistic;
 import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
+import org.apache.hadoop.metrics2.lib.MutableGaugeLong;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -33,7 +35,6 @@ import org.slf4j.LoggerFactory;
 import java.io.InputStream;
 
 import static org.apache.hadoop.fs.s3a.S3ATestUtils.*;
-import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.lookupGaugeStatistic;
 
 /**
  * Base class for scale tests; here is where the common scale configuration
@@ -183,15 +184,17 @@ public class S3AScaleTestBase extends AbstractS3ATestBase {
   }
 
   /**
-   * Get the gauge value of a statistic from the
-   * IOStatistics of the filesystem. Raises an assertion if
+   * Get the gauge value of a statistic. Raises an assertion if
    * there is no such gauge.
    * @param statistic statistic to look up
    * @return the value.
    */
   public long gaugeValue(Statistic statistic) {
-    return lookupGaugeStatistic(getFileSystem().getIOStatistics(),
-        statistic.getSymbol());
+    S3AInstrumentation instrumentation = getFileSystem().getInstrumentation();
+    MutableGaugeLong gauge = instrumentation.lookupGauge(statistic.getSymbol());
+    assertNotNull("No gauge " + statistic
+        + " in " + instrumentation.dump("", " = ", "\n", true), gauge);
+    return gauge.value();
   }
 
   /**