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

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

                Author: ASF GitHub Bot
            Created on: 25/Jul/22 10:17
            Start Date: 25/Jul/22 10:17
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on code in PR #4352:
URL: https://github.com/apache/hadoop/pull/4352#discussion_r928703332


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java:
##########
@@ -1351,6 +1351,7 @@ public boolean hasCapability(String capability) {
     case StreamCapabilities.READAHEAD:
     case StreamCapabilities.UNBUFFER:
     case StreamCapabilities.VECTOREDIO:
+    case StreamCapabilities.IOSTATISTICS_CONTEXT:

Review Comment:
   put it in alphabetical order



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextIntegration.java:
##########
@@ -116,13 +116,17 @@ public static IOStatisticsContext 
getCurrentIOStatisticsContext() {
   /**
    * Set the IOStatisticsContext for the current thread.
    * @param statisticsContext IOStatistics context instance for the
-   * current thread.
+   * current thread. If null, the context is reset.
    */
   public static void setThreadIOStatisticsContext(
       IOStatisticsContext statisticsContext) {
-    if (IS_THREAD_IOSTATS_ENABLED &&
-        ACTIVE_IOSTATS_CONTEXT.getForCurrentThread() != statisticsContext) {
-      ACTIVE_IOSTATS_CONTEXT.setForCurrentThread(statisticsContext);
+    if (IS_THREAD_IOSTATS_ENABLED) {

Review Comment:
   we should have a test to verify this doesn't trigger an NPE and removes the 
context
   
   test
   * get thread context
   * set the current context to null
   * get the thread context again
   * verify the ID values don't match, because a new context was generated



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextIntegration.java:
##########
@@ -152,4 +152,13 @@ public static IOStatisticsContext 
getThreadSpecificIOStatisticsContext(long test
     }
     return null;
   }
+
+  /**
+   * A method to enable IOStatisticsContext to override if set otherwise in
+   * the configurations for tests.
+   */
+  @VisibleForTesting
+  public static void enableIOStatisticsContext() {
+    IS_THREAD_IOSTATS_ENABLED = true;

Review Comment:
   how about, if it wasn't already true, log at info that it is being set



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContextIntegration.java:
##########
@@ -53,7 +53,7 @@ public final class IOStatisticsContextIntegration {
   /**
    * Is thread-level IO Statistics enabled?
    */
-  private static final boolean IS_THREAD_IOSTATS_ENABLED;
+  private static boolean IS_THREAD_IOSTATS_ENABLED;

Review Comment:
   needs to become lower case





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

    Worklog Id:     (was: 794813)
    Time Spent: 8h 40m  (was: 8.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: 8h 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.10#820010)

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

Reply via email to