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

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

mccormickt12 commented on PR #5322:
URL: https://github.com/apache/hadoop/pull/5322#issuecomment-1430524690

   > The change generally looks good to me.
   > 
   > One concern is that when `refetchLocation` is called inside 
`hedgedFetchBlockByteRange` we could remove a node from the ignoredList that is 
already part of the futures array. This would lead to multiple reads to the 
same node.
   > 
   > What I'm thinking of is
   > 
   > 1. Node A is added to futures array
   > 2. getFirstToComplete throws an InterruptedException when doing 
hedgedService.take()
   > 3. We call retchLocation, which remove node A from ignored list.
   > 4. The while loop re-adds Node A to the futures list.
   > 
   > I don't know if this actually can happen. Even if it is technically 
possible, it may not be an issue. Thoughts?
   
   yes @simbadzina I agree this was an issue. I've resolved this now. As you 
pointed out the future is removed in `getFirstToComplete`, so now we let it 
spin in the while loop, each time a future will be removed, and then once 
futures is empty and refetch is needed we will clear the ignore list




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

Reply via email to