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

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

                Author: ASF GitHub Bot
            Created on: 27/Jul/22 06:55
            Start Date: 27/Jul/22 06:55
    Worklog Time Spent: 10m 
      Work Description: mehakmeet opened a new pull request, #4639:
URL: https://github.com/apache/hadoop/pull/4639

   This adds a thread-level collector of IOStatistics, IOStatisticsContext,
   which can be:
   * Retrieved for a thread and cached for access from other
     threads.
   * reset() to record new statistics.
   * Queried for live statistics through the
     IOStatisticsSource.getIOStatistics() method.
   * Queries for a statistics aggregator for use in instrumented
     classes.
   * Asked to create a serializable copy in snapshot()
   
   The goal is to make it possible for applications with multiple
   threads performing different work items simultaneously
   to be able to collect statistics on the individual threads,
   and so generate aggregate reports on the total work performed
   for a specific job, query or similar unit of work.
   
   Some changes in IOStatistics-gathering classes are needed for 
   this feature
   * Caching the active context's aggregator in the object's
     constructor
   * Updating it in close()
   
   Slightly more work is needed in multithreaded code,
   such as the S3A committers, which collect statistics across
   all threads used in task and job commit operations.
   
   Currently the IOStatisticsContext-aware classes are:
   * The S3A input stream, output stream and list iterators.
   * RawLocalFileSystem's input and output streams.
   * The S3A committers.
   * The TaskPool class in hadoop-common, which propagates
     the active context into scheduled worker threads.
   
   Collection of statistics in the IOStatisticsContext
   is disabled process-wide by default until the feature 
   is considered stable.
   
   To enable the collection, set the option
   fs.thread.level.iostatistics.enabled
   to "true" in core-site.xml;
        
   Contributed by Mehakmeet Singh and Steve Loughran
   
   <!--
     Thanks for sending a pull request!
       1. If this is your first time, please read our contributor guidelines: 
https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
       2. Make sure your PR title starts with JIRA issue id, e.g., 
'HADOOP-17799. Your PR title ...'.
   -->
   
   ### Description of PR
   
   
   ### How was this patch tested?
   Region: `ap-south-1`
   All tests ran successfully.
   
   ### For code changes:
   
   - [X] Does the title or this PR starts with the corresponding JIRA issue id 
(e.g. 'HADOOP-17799. Your PR title ...')?
   - [X] Object storage: have the integration tests been executed and the 
endpoint declared according to the connector-specific documentation?
   - [X] If adding new dependencies to the code, are these dependencies 
licensed in a way that is compatible for inclusion under [ASF 
2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [X] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, 
`NOTICE-binary` files?
   
   




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

    Worklog Id:     (was: 795535)
    Time Spent: 10h 50m  (was: 10h 40m)

> 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: 10h 50m
>  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