huaxiang sun created HDFS-11819:
-----------------------------------

             Summary: HDFS client with hedged read, handle exceptions from 
callable  when the hedged read thread pool is exhausted
                 Key: HDFS-11819
                 URL: https://issues.apache.org/jira/browse/HDFS-11819
             Project: Hadoop HDFS
          Issue Type: Bug
    Affects Versions: 3.0.0-alpha2
            Reporter: huaxiang sun


When the hedged read thread pool is exhausted, the current behavior is that 
callable will be executed in the current thread context. The callable can throw 
out IOExceptions which is not handled and it will not start a 'hedged' read. 

https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L1131

Please see the following exception:
{code}
2017-05-11 22:42:35,883 WARN org.apache.hadoop.hdfs.BlockReaderFactory: I/O 
error constructing remote block reader.
org.apache.hadoop.net.ConnectTimeoutException: 3000 millis timeout while 
waiting for channel to be ready for connect. ch : 
java.nio.channels.SocketChannel[connection-pending remote=/*.*.*.*:50010]
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:533)
        at 
org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:3527)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:840)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:755)
        at 
org.apache.hadoop.hdfs.BlockReaderFactory.build(BlockReaderFactory.java:376)
        at 
org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1179)
        at 
org.apache.hadoop.hdfs.DFSInputStream.access$300(DFSInputStream.java:91)
        at 
org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1141)
        at 
org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1133)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at 
java.util.concurrent.ThreadPoolExecutor$CallerRunsPolicy.rejectedExecution(ThreadPoolExecutor.java:2022)
        at 
org.apache.hadoop.hdfs.DFSClient$2.rejectedExecution(DFSClient.java:3571)
        at 
java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823)
        at 
java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1369)
        at 
java.util.concurrent.ExecutorCompletionService.submit(ExecutorCompletionService.java:181)
        at 
org.apache.hadoop.hdfs.DFSInputStream.hedgedFetchBlockByteRange(DFSInputStream.java:1280)
        at org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1477)
        at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1439)
        at 
org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:92)
        at 
org.apache.hadoop.hbase.io.FileLink$FileLinkInputStream.read(FileLink.java:167)
        at 
org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:92)
        at 
org.apache.hadoop.hbase.io.hfile.HFileBlock.positionalReadWithExtra(HFileBlock.java:757)
        at 
org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1457)
        at 
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1682)
        at 
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1542)
        at 
org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:445)
        at 
org.apache.hadoop.hbase.util.CompoundBloomFilter.contains(CompoundBloomFilter.java:100)
        at 
org.apache.hadoop.hbase.regionserver.StoreFile$Reader.passesGeneralBloomFilter(StoreFile.java:1383)
        at 
org.apache.hadoop.hbase.regionserver.StoreFile$Reader.passesBloomFilter(StoreFile.java:1247)
        at 
org.apache.hadoop.hbase.regionserver.StoreFileScanner.shouldUseScanner(StoreFileScanner.java:469)
        at 
org.apache.hadoop.hbase.regionserver.StoreScanner.selectScannersFrom(StoreScanner.java:393)
        at 
org.apache.hadoop.hbase.regionserver.StoreScanner.getScannersNoCompaction(StoreScanner.java:312)
        at 
org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:192)
        at 
org.apache.hadoop.hbase.regionserver.HStore.createScanner(HStore.java:2106)
        at 
org.apache.hadoop.hbase.regionserver.HStore.getScanner(HStore.java:2096)
        at 
org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.<init>(HRegion.java:5544)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.instantiateRegionScanner(HRegion.java:2569)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2555)
        at 
org.apache.hadoop.hbase.regionserver.HRegion.getScanner(HRegion.java:2536)
        at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6791)
        at org.apache.hadoop.hbase.regionserver.HRegion.get(HRegion.java:6770)
        at 
org.apache.hadoop.hbase.regionserver.RSRpcServices.get(RSRpcServices.java:2025)
        at 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:33644)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2170)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:109)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:185)
        at 
org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:165)

{code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-dev-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-dev-h...@hadoop.apache.org

Reply via email to