Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 20C0B200C86 for ; Wed, 17 May 2017 03:39:11 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 1F4A2160BCF; Wed, 17 May 2017 01:39:11 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 3ECF5160BC1 for ; Wed, 17 May 2017 03:39:10 +0200 (CEST) Received: (qmail 34675 invoked by uid 500); 17 May 2017 01:39:08 -0000 Mailing-List: contact hdfs-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list hdfs-issues@hadoop.apache.org Received: (qmail 34664 invoked by uid 99); 17 May 2017 01:39:08 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 17 May 2017 01:39:08 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 6C4C31AF923 for ; Wed, 17 May 2017 01:39:08 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.201 X-Spam-Level: X-Spam-Status: No, score=-99.201 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id avEwJD6_gbWM for ; Wed, 17 May 2017 01:39:06 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 0B9F95FC7F for ; Wed, 17 May 2017 01:39:06 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 0635AE0D42 for ; Wed, 17 May 2017 01:39:05 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 5069521943 for ; Wed, 17 May 2017 01:39:04 +0000 (UTC) Date: Wed, 17 May 2017 01:39:04 +0000 (UTC) From: "huaxiang sun (JIRA)" To: hdfs-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HDFS-11819) HDFS client with hedged read, handle exceptions from callable when the hedged read thread pool is exhausted MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 17 May 2017 01:39:11 -0000 [ https://issues.apache.org/jira/browse/HDFS-11819?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16013362#comment-16013362 ] huaxiang sun commented on HDFS-11819: ------------------------------------- Update after more log reading, so when the hedged read thread pool is exhausted, the hedged read does not work anymore. Since the thread is not interrupted, it is easy to exhaust the thread pool in some error conditions. https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L1228 Discussed with [~atm] and [~jzhuge], they will follow up with the new jira if it can be improved after research. I am closing this one as invalid. > 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 > Assignee: 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.(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.(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-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org