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

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

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


##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java:
##########
@@ -374,6 +376,16 @@ public class S3AFileSystem extends FileSystem implements 
StreamCapabilities,
    */
   private ArnResource accessPoint;
 
+  /**
+   * Does the fs have thread-level IOStats support enabled?
+   */
+  private boolean isThreadLevelIOStatsEnabled;

Review Comment:
   as discussed, this should go into hadoop common



##########
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java:
##########
@@ -444,6 +450,9 @@ public void close() throws IOException {
    */
   private synchronized void cleanupOnClose() {
     cleanupWithLogger(LOG, getActiveBlock(), blockFactory);
+    if(ioStatisticsContext != null) {

Review Comment:
   nit, add a space



##########
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:
   yes



##########
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsContext.java:
##########
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Context used to capture IOStatistics per thread in a task.
+ */
+public interface IOStatisticsContext {
+
+  IOStatisticsSnapshot getThreadIOStatistics();

Review Comment:
   nit: javadocs





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

    Worklog Id:     (was: 782506)
    Time Spent: 1.5h  (was: 1h 20m)

> 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: 1.5h
>  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-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to