浏览代码

HADOOP-13735 ITestS3AFileContextStatistics.testStatistics() failing. Contributed by Pieter Reuse

Steve Loughran 8 年之前
父节点
当前提交
9ae270af02

+ 1 - 1
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java

@@ -548,7 +548,7 @@ public class S3AFileSystem extends FileSystem {
               progress,
               partSize,
               blockFactory,
-              instrumentation.newOutputStreamStatistics(),
+              instrumentation.newOutputStreamStatistics(statistics),
               new WriteOperationHelper(key)
           ),
           null);

+ 10 - 3
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java

@@ -37,6 +37,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.hadoop.fs.FileSystem.Statistics;
 
 import static org.apache.hadoop.fs.s3a.Statistic.*;
 
@@ -639,9 +640,8 @@ public class S3AInstrumentation {
    * Create a stream output statistics instance.
    * @return the new instance
    */
-
-  OutputStreamStatistics newOutputStreamStatistics() {
-    return new OutputStreamStatistics();
+  OutputStreamStatistics newOutputStreamStatistics(Statistics statistics) {
+    return new OutputStreamStatistics(statistics);
   }
 
   /**
@@ -677,6 +677,12 @@ public class S3AInstrumentation {
     private final AtomicLong queueDuration = new AtomicLong(0);
     private final AtomicLong exceptionsInMultipartFinalize = new AtomicLong(0);
 
+    private Statistics statistics;
+
+    public OutputStreamStatistics(Statistics statistics){
+      this.statistics = statistics;
+    }
+
     /**
      * Block is queued for upload.
      */
@@ -717,6 +723,7 @@ public class S3AInstrumentation {
     /** Intermediate report of bytes uploaded. */
     void bytesTransferred(long byteCount) {
       bytesUploaded.addAndGet(byteCount);
+      statistics.incrementBytesWritten(byteCount);
       bytesPendingUpload.addAndGet(-byteCount);
       incrementGauge(STREAM_WRITE_BLOCK_UPLOADS_DATA_PENDING, -byteCount);
     }