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

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

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


##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsContext.java:
##########
@@ -51,21 +51,21 @@ public class IOStatisticsContext {
   private static final boolean IS_THREAD_IOSTATS_ENABLED;
 
   private static final WeakReferenceThreadMap<IOStatisticsContext>
-      ACTIVE_IOSTATS_CONTEXT = new WeakReferenceThreadMap<>(
-      IOStatisticsContext::createNewInstance,
-      IOStatisticsContext::referenceLostContext
+      ACTIVE_IOSTATS_CONTEXT =
+      new WeakReferenceThreadMap<>(IOStatisticsContext::createNewInstance,
+          IOStatisticsContext::referenceLostContext
   );
 
   /**
    * Collecting IOStatistics per thread.
    */
   private final WeakReferenceThreadMap<IOStatisticsAggregator>

Review Comment:
   Okay, I'll do that. My thinking behind this was to also have a way to get 
EmptyIOstatisticsSource as one of the implementations we could return for a 
thread ID, which would have a no-op for aggregation, but thinking this through, 
it seems like having this thread map solely for "enabled" thread IOStats makes 
more sense and we can return the EmptyIOstatisticsSource, while getting the 
IOstats without setting it in. Makes resetting easier too. 





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

    Worklog Id:     (was: 789485)
    Time Spent: 3h  (was: 2h 50m)

> 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: 3h
>  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