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

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

                Author: ASF GitHub Bot
            Created on: 21/Jul/22 10:38
            Start Date: 21/Jul/22 10:38
    Worklog Time Spent: 10m 
      Work Description: steveloughran commented on PR #4352:
URL: https://github.com/apache/hadoop/pull/4352#issuecomment-1191326991

   This is my draft commit message btw
   
   ----
   
   
   Adds a new IOStatistics class IOStatisticsContext.
   
   This is the active collector of thread-level statistics for
   The current thread.
   
   The S3A Filesystem's input and output streams, and listing
   operations, all update this context when close() is called on
   them (and not before!), so there is effectively automatic
   aggregation of all IO statistics performed by a single thread.
   
   The IOStatisticsContext of a thread can be retrieved and
   cached for invocation in other threads. Holding such a
   reference also ensures that the context will not be garbage
   collected.
   
   To collect statistics on a thread:
   
   1. Retrieve the active context with a call to
      IOStatisticsContext.getCurrentIOStatisticsContext()
   2. Call IOStatisticsContext.reset() to reset all statistics
   3. Call getIOStatistics() on it for the latest values, or
      snapshot() for a snapshot of them.
         
   To instrument filesystem objects for thread-level
   IOStatistics
   
   1. Cache the current IOStatisticsContext context or just its
      aggregator in the object constructor.
   2. In the close() operation, aggregate() the object's own statistics.
   3. Pass the context into worker threads performing work
      on behalf of this thread, through
         IOStatisticsContext.setThreadIOStatisticsContext();
         set it to null afterwards.
   
   TaskPool does the context propagation and reset
   automatically.
   
   Contributed by Mehakmeet Singh
   
   ---




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

    Worklog Id:     (was: 793670)
    Time Spent: 7h 40m  (was: 7.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: 7h 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