See https://issues.apache.org/jira/browse/HDFS-13571.
The problem for HBase is that the region server will keep a large amount of DFSInputStream open, for random access. If a DN node is completely gone, i.e, can not be reached through the network, you will face a connection timeout, usually it will be very slow, in your case, 60 seconds, for HBase to find out the connection is broken. It is not easy for HBase to fix this problem and in HDFS-13571, the hdfs community introduced a way to mark a DN as dead before sending request to it, and the dead node information is shared through all DFSInputStream so after the first time you see a connection timeout of a DN, soon all the DFSInputStream will give up requesting the dead DN, which could mostly solve the above problem. You can try to build HBase with hadoop 3.3.x and deploy it again for fixing the problem, as the code changes are mostly at client side. Thanks. Miao Wang <[email protected]> 于2023年10月30日周一 18:49写道: > > Hi, community > > > We have an HBase cluster using version 2.4.6 and Hadoop version 3.0.0 > cdh6.3.2,I have two questions to ask. Could you please answer them? Thank you > > > 1.When datanode crashes for an extended period of time and the client fails > to hit the cache when requesting data, the region server will still request > abnormal datanode nodes. Is this cached information only updated when the > table is making a major? ,Is the information in this cache only updated when > the table generates the main table? When the dn service encounters an > exception, how can I proactively update this information in the regional > server cache? > The error message is as follows: > > > Failed to connect to /10.11.34.29:9866 for file > /hbase/data/default/tj_idcard_sha_02/429606d2083c287a13d0ebe43774938f/cf/dc063bb106b34039a014690f86b1c22c > for block > BP-1561188726-10.11.39.10-1684756341563:blk_1078186539_4445860:java.io.IOException: > Connection reset by peer > > > 2.When datanoe encounters an exception, the regionserve will generate the > following error log, which seriously affects the client's request time. > Analyzing the log roughly affects the duration of ten minutes. Why are these > two error messages still inconsistent? How can I avoid it? > > > > > Connection failure: Failed to connect to /10.11.34.29:9866 for file > /hbase/archive/data/default/tj_soical_v05_feature_hb_rhsh/500ea510085d36ce0efa33ac63a15a88/cf/e6003a114c5d415ba720ac7657538120 > for block > BP-1561188726-10.11.39.10-1684756341563:blk_1078131727_4391047:java.net.SocketTimeoutException: > 60000 millis timeout while waiting for channel to be ready for read. ch : > java.nio.channels.SocketChannel[connected local=/10.11.2.142:14154 > remote=/10.11.34.29:9866] > java.net.SocketTimeoutException: 60000 millis timeout while waiting for > channel to be ready for read. ch : java.nio.channels.SocketChannel[connected > local=/10.11.2.142:14154 remote=/10.11.34.29:9866] > at sun.nio.ch.FileDispatcherImpl.read0(Native Method) > at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39) > at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) > at sun.nio.ch.IOUtil.read(IOUtil.java:197) > at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380) > at > org.apache.hadoop.net.SocketInputStream$Reader.performIO(SocketInputStream.java:57) > at > org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) > at > org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118) > at java.io.FilterInputStream.read(FilterInputStream.java:83) > at > org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:537) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderRemote.newBlockReader(BlockReaderRemote.java:407) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReader(BlockReaderFactory.java:848) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:744) > at > org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:379) > at > org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:645) > at > org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1050) > at > org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1002) > at > org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1361) > at > org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1325) > at > org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:93) > at > org.apache.hadoop.hbase.io.util.BlockIOUtils.preadWithExtra(BlockIOUtils.java:214) > at > org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readAtOffset(HFileBlock.java:1518) > at > org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal(HFileBlock.java:1741) > at > org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData(HFileBlock.java:1550) > at > org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock(HFileReaderImpl.java:1337) > at > org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter.getBloomBlock(CompoundBloomFilter.java:122) > at > org.apache.hadoop.hbase.io.hfile.CompoundBloomFilter.contains(CompoundBloomFilter.java:102) > at > org.apache.hadoop.hbase.regionserver.StoreFileReader.checkGeneralBloomFilter(StoreFileReader.java:433) > at > org.apache.hadoop.hbase.regionserver.StoreFileReader.passesGeneralRowBloomFilter(StoreFileReader.java:322) > at > org.apache.hadoop.hbase.regionserver.StoreFileReader.passesBloomFilter(StoreFileReader.java:251) > at > org.apache.hadoop.hbase.regionserver.StoreFileScanner.shouldUseScanner(StoreFileScanner.java:491) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.selectScannersFrom(StoreScanner.java:472) > at > org.apache.hadoop.hbase.regionserver.StoreScanner.<init>(StoreScanner.java:249)
