[ 
https://issues.apache.org/jira/browse/HDFS-16896?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17692337#comment-17692337
 ] 

ASF GitHub Bot commented on HDFS-16896:
---------------------------------------

mkuchenbecker commented on code in PR #5322:
URL: https://github.com/apache/hadoop/pull/5322#discussion_r1114855356


##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java:
##########
@@ -197,6 +197,15 @@ private void clearLocalDeadNodes() {
     deadNodes.clear();
   }
 
+  /**
+   * Clear list of ignored nodes used for hedged reads.
+   */
+  private void clearIgnoredNodes(Collection<DatanodeInfo> ignoredNodes) {

Review Comment:
   Nit. Param documentation. 



##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java:
##########
@@ -1337,8 +1352,12 @@ private void hedgedFetchBlockByteRange(LocatedBlock 
block, long start,
         } catch (InterruptedException ie) {
           // Ignore and retry
         }
-        if (refetch) {
-          refetchLocations(block, ignored);
+        // If refetch is true, then all nodes are in deadNodes or ignoredNodes.
+        // We should loop through all futures and remove them, so we do not
+        // have concurrent requests to the same node.
+        // Once all futures are cleared, we can clear the ignoredNodes and 
retry.

Review Comment:
   Ignored and dead nodes are cleared, correct?



##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java:
##########
@@ -224,7 +233,7 @@ boolean deadNodesContain(DatanodeInfo nodeInfo) {
   }
 
   /**
-   * Grab the open-file info from namenode
+   * Grab the open-file info from namenode.

Review Comment:
   Is this change needed?



##########
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java:
##########
@@ -197,6 +197,15 @@ private void clearLocalDeadNodes() {
     deadNodes.clear();
   }
 
+  /**
+   * Clear list of ignored nodes used for hedged reads.
+   */
+  private void clearIgnoredNodes(Collection<DatanodeInfo> ignoredNodes) {

Review Comment:
   Does it make more sense to clean up ignored / dead in the same function so 
we don't need to worry about calling both?



##########
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java:
##########
@@ -603,7 +603,9 @@ public Void answer(InvocationOnMock invocation) throws 
Throwable {
       input.read(0, buffer, 0, 1024);
       Assert.fail("Reading the block should have thrown 
BlockMissingException");
     } catch (BlockMissingException e) {
-      assertEquals(3, input.getHedgedReadOpsLoopNumForTesting());
+      // The result of 9 is due to 2 blocks by 4 iterations plus one because
+      // hedgedReadOpsLoopNumForTesting is incremented at start of the loop.
+      assertEquals(9, input.getHedgedReadOpsLoopNumForTesting());

Review Comment:
   We are tripling the IO per hedged request? 





> HDFS Client hedged read has increased failure rate than without hedged read
> ---------------------------------------------------------------------------
>
>                 Key: HDFS-16896
>                 URL: https://issues.apache.org/jira/browse/HDFS-16896
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: hdfs-client
>            Reporter: Tom McCormick
>            Assignee: Tom McCormick
>            Priority: Major
>              Labels: pull-request-available
>
> When hedged read is enabled by HDFS client, we see an increased failure rate 
> on reads.
> *stacktrace*
>  
> {code:java}
> Caused by: org.apache.hadoop.hdfs.BlockMissingException: Could not obtain 
> block: BP-1183972111-10.197.192.88-1590025572374:blk_17114848218_16043459722 
> file=/data/tracking/streaming/AdImpressionEvent/daily/2022/07/18/compaction_1/part-r-1914862.1658217125623.1362294472.orc
> at 
> org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1077)
> at 
> org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1060)
> at 
> org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1039)
> at 
> org.apache.hadoop.hdfs.DFSInputStream.hedgedFetchBlockByteRange(DFSInputStream.java:1365)
> at org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1572)
> at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1535)
> at org.apache.hadoop.fs.FSInputStream.readFully(FSInputStream.java:121)
> at 
> org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:112)
> at 
> org.apache.hadoop.fs.RetryingInputStream.lambda$readFully$3(RetryingInputStream.java:172)
> at org.apache.hadoop.fs.RetryPolicy.lambda$run$0(RetryPolicy.java:137)
> at org.apache.hadoop.fs.NoOpRetryPolicy.run(NoOpRetryPolicy.java:36)
> at org.apache.hadoop.fs.RetryPolicy.run(RetryPolicy.java:136)
> at 
> org.apache.hadoop.fs.RetryingInputStream.readFully(RetryingInputStream.java:168)
> at 
> org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:112)
> at 
> org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:112)
> at 
> io.trino.plugin.hive.orc.HdfsOrcDataSource.readInternal(HdfsOrcDataSource.java:76)
> ... 46 more
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to