You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/06/09 18:38:00 UTC

[jira] [Work logged] (HADOOP-17461) Add thread-level IOStatistics Context

     [ https://issues.apache.org/jira/browse/HADOOP-17461?focusedWorklogId=780090&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-780090 ]

ASF GitHub Bot logged work on HADOOP-17461:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 09/Jun/22 18:37
            Start Date: 09/Jun/22 18:37
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on code in PR #4352:
URL: https://github.com/apache/hadoop/pull/4352#discussion_r893822412


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextImpl.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.WeakReferenceThreadMap;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+
+/**
+ * Implementing the IOStatisticsContext interface.
+ */
+public class IOStatisticsContextImpl implements IOStatisticsContext {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(IOStatisticsContextImpl.class);
+
+  /**
+   * Collecting IOStatistics per thread.
+   */
+  private final WeakReferenceThreadMap<IOStatisticsSnapshot>
+      threadIOStatsContext = new WeakReferenceThreadMap<>(
+      this::getIOStatisticsSnapshotFactory,

Review Comment:
   needs indentation



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextImpl.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.WeakReferenceThreadMap;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+
+/**
+ * Implementing the IOStatisticsContext interface.
+ */
+public class IOStatisticsContextImpl implements IOStatisticsContext {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(IOStatisticsContextImpl.class);
+
+  /**
+   * Collecting IOStatistics per thread.
+   */
+  private final WeakReferenceThreadMap<IOStatisticsSnapshot>
+      threadIOStatsContext = new WeakReferenceThreadMap<>(
+      this::getIOStatisticsSnapshotFactory,
+      this::referenceLost);
+
+  /**
+   * A Method to act as an IOStatisticsSnapshot factory, in a
+   * WeakReferenceThreadMap.
+   *
+   * @param key ThreadID.
+   * @return an Instance of IOStatisticsSnapshot.
+   */
+  private IOStatisticsSnapshot getIOStatisticsSnapshotFactory(Long key) {
+    return new IOStatisticsSnapshot();
+  }
+
+  /**
+   * In case of reference loss.
+   *
+   * @param key ThreadID.
+   */
+  private void referenceLost(Long key) {
+    LOG.info("Reference lost for threadID: {}", key);

Review Comment:
   maybe debug



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextImpl.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.WeakReferenceThreadMap;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+
+/**
+ * Implementing the IOStatisticsContext interface.
+ */
+public class IOStatisticsContextImpl implements IOStatisticsContext {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(IOStatisticsContextImpl.class);
+
+  /**
+   * Collecting IOStatistics per thread.
+   */
+  private final WeakReferenceThreadMap<IOStatisticsSnapshot>
+      threadIOStatsContext = new WeakReferenceThreadMap<>(
+      this::getIOStatisticsSnapshotFactory,
+      this::referenceLost);
+
+  /**
+   * A Method to act as an IOStatisticsSnapshot factory, in a
+   * WeakReferenceThreadMap.
+   *
+   * @param key ThreadID.
+   * @return an Instance of IOStatisticsSnapshot.
+   */
+  private IOStatisticsSnapshot getIOStatisticsSnapshotFactory(Long key) {
+    return new IOStatisticsSnapshot();
+  }
+
+  /**
+   * In case of reference loss.
+   *
+   * @param key ThreadID.
+   */
+  private void referenceLost(Long key) {
+    LOG.info("Reference lost for threadID: {}", key);
+  }
+
+  /**
+   * A Method to get the IOStatisticsSnapshot of the currentThread. This
+   * denotes the aggregated IOStatistics per thread.
+   *

Review Comment:
   comment that it is not thread safe as it doesn't need to be



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextImpl.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.WeakReferenceThreadMap;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+
+/**
+ * Implementing the IOStatisticsContext interface.
+ */
+public class IOStatisticsContextImpl implements IOStatisticsContext {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(IOStatisticsContextImpl.class);
+
+  /**
+   * Collecting IOStatistics per thread.
+   */
+  private final WeakReferenceThreadMap<IOStatisticsSnapshot>
+      threadIOStatsContext = new WeakReferenceThreadMap<>(
+      this::getIOStatisticsSnapshotFactory,
+      this::referenceLost);
+
+  /**
+   * A Method to act as an IOStatisticsSnapshot factory, in a
+   * WeakReferenceThreadMap.
+   *
+   * @param key ThreadID.
+   * @return an Instance of IOStatisticsSnapshot.
+   */
+  private IOStatisticsSnapshot getIOStatisticsSnapshotFactory(Long key) {
+    return new IOStatisticsSnapshot();
+  }
+
+  /**
+   * In case of reference loss.
+   *
+   * @param key ThreadID.
+   */
+  private void referenceLost(Long key) {
+    LOG.info("Reference lost for threadID: {}", key);
+  }
+
+  /**
+   * A Method to get the IOStatisticsSnapshot of the currentThread. This
+   * denotes the aggregated IOStatistics per thread.
+   *
+   * @return the instance of IOStatisticsSnapshot.
+   */
+  @Override
+  public IOStatisticsSnapshot getThreadIOStatistics() {
+    IOStatisticsSnapshot ioStatisticsSnapshot =
+        threadIOStatsContext.getForCurrentThread();
+    if (ioStatisticsSnapshot == null) {
+      // Create and set IOStatsSnapshot in the ThreadPool.
+      ioStatisticsSnapshot = new IOStatisticsSnapshot();
+      threadIOStatsContext.setForCurrentThread(ioStatisticsSnapshot);
+    }
+    // If an instance is present return it.
+    return ioStatisticsSnapshot;
+  }
+
+  @Override
+  public String toString() {
+    return getThreadIOStatistics().toString();

Review Comment:
   this is potentially expensive and give different values when called in different threads. best cut



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -670,6 +685,19 @@ public Statistics getInstanceStatistics() {
             return S3AFileSystem.this.statistics;
           }
         });
+    // If Thread-level IOStats support is enabled, initialize the context.
+    // This would initialize a ThreadLocal Instance.
+    if(isIOStatsContextEnabled()) {
+      this.ioStatisticsContext = new IOStatisticsContextImpl();
+    }
+  }
+
+  /**
+   *

Review Comment:
   nit: javadocs



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextImpl.java:
##########
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.statistics.impl;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.impl.WeakReferenceThreadMap;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot;
+
+/**
+ * Implementing the IOStatisticsContext interface.
+ */
+public class IOStatisticsContextImpl implements IOStatisticsContext {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(IOStatisticsContextImpl.class);
+
+  /**
+   * Collecting IOStatistics per thread.
+   */
+  private final WeakReferenceThreadMap<IOStatisticsSnapshot>
+      threadIOStatsContext = new WeakReferenceThreadMap<>(
+      this::getIOStatisticsSnapshotFactory,
+      this::referenceLost);
+
+  /**
+   * A Method to act as an IOStatisticsSnapshot factory, in a
+   * WeakReferenceThreadMap.
+   *
+   * @param key ThreadID.
+   * @return an Instance of IOStatisticsSnapshot.
+   */
+  private IOStatisticsSnapshot getIOStatisticsSnapshotFactory(Long key) {
+    return new IOStatisticsSnapshot();
+  }
+
+  /**
+   * In case of reference loss.
+   *
+   * @param key ThreadID.
+   */
+  private void referenceLost(Long key) {
+    LOG.info("Reference lost for threadID: {}", key);
+  }
+
+  /**
+   * A Method to get the IOStatisticsSnapshot of the currentThread. This

Review Comment:
   nit: space between current and thread



##########
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/LambdaTestUtils.java:
##########
@@ -65,6 +66,15 @@ private LambdaTestUtils() {
    */
   public static final String NULL_RESULT = "(null)";
 
+  /**
+   * Atomic references to be used to re-throw an Exception or an ASE
+   * caught inside a lambda function.
+   */
+  public static final AtomicReference<Exception> futureExcp =

Review Comment:
   1. not sure that this is the right place for what's been a fairly stable class for a while
   1. give full title in their names, upper case
   2. make private



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java:
##########
@@ -561,6 +566,12 @@ public synchronized void close() throws IOException {
       try {
         // close or abort the stream; blocking
         awaitFuture(closeStream("close() operation", false, true));
+        // Collect ThreadLevel IOStats
+        if (ioStatisticsContext != null) {
+          ioStatisticsContext.getThreadIOStatistics().aggregate(streamStatistics.getIOStatistics());

Review Comment:
   ioStatisticsContext.getThreadIOStatistics() needs to be cached in the constructor, so that even when closed *in a different thread* the original stat context is updated. this also guarantees that the reference doesn't get cleaned up while a stream is open



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.THREAD_LEVEL_IOSTATS_ENABLED;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.test.LambdaTestUtils.maybeReThrowFutureASE;
+import static org.apache.hadoop.test.LambdaTestUtils.maybeReThrowFutureException;
+import static org.apache.hadoop.test.LambdaTestUtils.setFutureASE;
+import static org.apache.hadoop.test.LambdaTestUtils.setFutureException;
+
+public class ITestS3AIOStatisticsContext extends AbstractS3ATestBase {

Review Comment:
   1. add javadoc of test purpose
   
   2. add tests to verify the thread used in the contructor has its stats updated for both input and output io. open in the junit thread, do work and close in others. assert on the junit thread values



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java:
##########
@@ -77,11 +80,13 @@ public S3AReadOpContext(
       Invoker invoker,
       @Nullable FileSystem.Statistics stats,
       S3AStatisticsContext instrumentation,
-      FileStatus dstFileStatus) {
+      FileStatus dstFileStatus,
+      final IOStatisticsContext ioStatisticsContext) {
 
     super(invoker, stats, instrumentation,
         dstFileStatus);
     this.path = requireNonNull(path);
+    this.ioStatisticsContext = ioStatisticsContext;

Review Comment:
   can this be null? prefer not



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -193,6 +198,7 @@ class S3ABlockOutputStream extends OutputStream implements
       initMultipartUpload();
     }
     this.isCSEEnabled = builder.isCSEEnabled;
+    this.ioStatisticsContext = builder.ioStatisticsContext;

Review Comment:
   cache the thread iostats here and use in close(), something like `threadIOStatistics`



##########
hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AIOStatisticsContext.java:
##########
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.s3a;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+
+import org.junit.Test;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.statistics.IOStatisticAssertions;
+import org.apache.hadoop.fs.statistics.IOStatistics;
+import org.apache.hadoop.fs.statistics.IOStatisticsContext;
+import org.apache.hadoop.fs.statistics.StreamStatisticNames;
+import org.apache.hadoop.util.concurrent.HadoopExecutors;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
+import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset;
+import static org.apache.hadoop.fs.s3a.Constants.THREAD_LEVEL_IOSTATS_ENABLED;
+import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides;
+import static org.apache.hadoop.test.LambdaTestUtils.maybeReThrowFutureASE;
+import static org.apache.hadoop.test.LambdaTestUtils.maybeReThrowFutureException;
+import static org.apache.hadoop.test.LambdaTestUtils.setFutureASE;
+import static org.apache.hadoop.test.LambdaTestUtils.setFutureException;
+
+public class ITestS3AIOStatisticsContext extends AbstractS3ATestBase {
+
+  private static final int SMALL_THREADS = 2;
+  private static final int READ_BYTES_FIRST = 100;
+  private static final int READ_BYTES_SECOND = 50;
+
+  @Override
+  protected Configuration createConfiguration() {
+    Configuration configuration = super.createConfiguration();
+    removeBaseAndBucketOverrides(configuration, THREAD_LEVEL_IOSTATS_ENABLED);
+    return configuration;
+  }
+
+  /**
+   * Verify that S3AInputStream aggregates per thread IOStats collection
+   * correctly.
+   */
+  @Test
+  public void testS3AInputStreamIOStatisticsContext()
+      throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Path path = path(getMethodName());
+    byte[] data = dataset(256, 'a', 'z');
+    byte[] readDataFirst = new byte[READ_BYTES_FIRST];
+    byte[] readDataSecond = new byte[READ_BYTES_SECOND];
+    writeDataset(fs, path, data, data.length, 1024, true);
+
+    final ExecutorService executor =
+        HadoopExecutors.newFixedThreadPool(SMALL_THREADS);
+    CountDownLatch latch = new CountDownLatch(SMALL_THREADS);
+
+    try {
+      for (int i = 0; i < SMALL_THREADS; i++) {
+        executor.submit(() -> {
+          try {
+            IOStatistics ioStatisticsFirst;
+            try (FSDataInputStream in = fs.open(path)) {
+              in.seek(50);
+              in.read(readDataFirst);
+              in.close();
+              ioStatisticsFirst = assertThreadStatisticsBytesRead(in,
+                  READ_BYTES_FIRST);
+            }
+            // Stream is closed for a thread. Re-open and do more operations.
+            IOStatistics ioStatisticsSecond;
+            try (FSDataInputStream in = fs.open(path)) {
+              in.seek(100);
+              in.read(readDataSecond);
+              in.close();
+              ioStatisticsSecond = assertThreadStatisticsBytesRead(in,
+                  READ_BYTES_FIRST + READ_BYTES_SECOND);
+            }
+            latch.countDown();
+          } catch (Exception e) {
+            latch.countDown();
+            setFutureException(e);
+            LOG.error("An error occurred while doing a task in the thread", e);
+          } catch (AssertionError ase) {
+            latch.countDown();
+            setFutureASE(ase);
+            throw ase;
+          }
+        });
+      }
+      // wait for tasks to finish.
+      latch.await();
+    } finally {
+      executor.shutdown();
+    }
+
+    // Check if an Excp or ASE was caught while the test threads were running.
+    maybeReThrowFutureException();
+    maybeReThrowFutureASE();
+
+  }
+
+  @Test
+  public void testS3ABlockOutputStreamIOStatisticsContext()
+      throws Exception {
+    S3AFileSystem fs = getFileSystem();
+    Path path = path(getMethodName());
+    byte[] data = dataset(256, 'a', 'z');
+    byte[] writeDataFirst = new byte[READ_BYTES_FIRST];
+    byte[] writeDataSecond = new byte[READ_BYTES_SECOND];
+    writeDataset(fs, path, data, data.length, 1024, true);
+
+    final ExecutorService executor =
+        HadoopExecutors.newFixedThreadPool(SMALL_THREADS);
+    CountDownLatch latch = new CountDownLatch(SMALL_THREADS);
+
+    try {
+      for (int i = 0; i < SMALL_THREADS; i++) {
+        executor.submit(() -> {
+          try {
+            IOStatistics ioStatisticsFirst;
+            try (FSDataOutputStream out = fs.create(path)) {
+              out.write(writeDataFirst);
+              out.close();
+              ioStatisticsFirst = assertThreadStatisticsBytesWrite(out,
+                  READ_BYTES_FIRST);
+            }
+            // Stream is closed for a thread. Re-open and do more operations.
+            IOStatistics ioStatisticsSecond;
+            try (FSDataOutputStream out = fs.create(path)) {
+              out.write(writeDataSecond);
+              out.close();
+              ioStatisticsSecond = assertThreadStatisticsBytesWrite(out,
+                  READ_BYTES_FIRST + READ_BYTES_SECOND);
+            }
+            latch.countDown();
+          } catch (Exception e) {
+            latch.countDown();
+            setFutureException(e);
+            LOG.error("An error occurred while doing a task in the thread", e);
+          } catch (AssertionError ase) {
+            latch.countDown();
+            setFutureASE(ase);
+            throw ase;
+          }
+        });
+      }
+      // wait for tasks to finish.
+      latch.await();
+    } finally {
+      executor.shutdown();
+    }
+
+    // Check if an Excp or ASE was caught while the test threads were running.
+    maybeReThrowFutureException();
+    maybeReThrowFutureASE();
+
+  }
+
+  private IOStatistics assertThreadStatisticsBytesWrite(FSDataOutputStream out, int writeBytes) {
+    S3ABlockOutputStream s3aOut = (S3ABlockOutputStream) out.getWrappedStream();
+    IOStatisticsContext ioStatisticsContext = s3aOut.getIoStatisticsContext();

Review Comment:
   stream should just cache the thread IOStatistics in its constructor, not the context.
   
   



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java:
##########
@@ -561,6 +566,12 @@ public synchronized void close() throws IOException {
       try {
         // close or abort the stream; blocking
         awaitFuture(closeStream("close() operation", false, true));
+        // Collect ThreadLevel IOStats
+        if (ioStatisticsContext != null) {
+          ioStatisticsContext.getThreadIOStatistics().aggregate(streamStatistics.getIOStatistics());
+          LOG.debug("IOStatistics of thread: {}\n{}",
+              Thread.currentThread().getId(), ioStatisticsContext.toString());

Review Comment:
   too expensive. 
   Use IOStatisticsLogging for on-demand logging of the thread io stats you'll be caching





Issue Time Tracking
-------------------

    Worklog Id:     (was: 780090)
    Time Spent: 40m  (was: 0.5h)

> Add thread-level IOStatistics Context
> -------------------------------------
>
>                 Key: HADOOP-17461
>                 URL: https://issues.apache.org/jira/browse/HADOOP-17461
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs, fs/azure, fs/s3
>    Affects Versions: 3.3.1
>            Reporter: Steve Loughran
>            Assignee: Mehakmeet Singh
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> For effective reporting of the iostatistics of individual worker threads, we need a thread-level context which IO components update.
> * this contact needs to be passed in two background thread forming work on behalf of a task.
> * IO Components (streams, iterators, filesystems) need to update this context statistics as they perform work
> * Without double counting anything.
> I imagine a ThreadLocal IOStatisticContext which will be updated in the FileSystem API Calls. This context MUST be passed into the background threads used by a task, so that IO is correctly aggregated.
> I don't want streams, listIterators &c to do the updating as there is more risk of double counting. However, we need to see their statistics if we want to know things like "bytes discarded in backwards seeks". And I don't want to be updating a shared context object on every read() call.
> If all we want is store IO (HEAD, GET, DELETE, list performance etc) then the FS is sufficient. 
> If we do want the stream-specific detail, then I propose
> * caching the context in the constructor
> * updating it only in close() or unbuffer() (as we do from S3AInputStream to S3AInstrumenation)
> * excluding those we know the FS already collects.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org