Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-16332

Expired block token causes slow read due to missing handling in sasl handshake

    XMLWordPrintableJSON

Details

    Description

      We're operating the HBase 1.4.x cluster on Hadoop 2.8.5.
      We're recently evaluating Kerberos secured HBase and Hadoop cluster with production load and we observed HBase's response slows >= several seconds, and about several minutes for worst-case (about once~three times a month).

      The following image is a scatter plot of HBase's response slow, each circle is each base's slow response log.
      The X-axis is the date time of the log occurred, the Y-axis is the response slow time.

      We could reproduce this issue by reducing "dfs.block.access.token.lifetime" and we could figure out the cause.
      (We used dfs.block.access.token.lifetime=60, i.e. 1 hour)

      When hedged read enabled:

      When hedged read disabled:

      As you can see, it's worst if the hedged read is enabled. However, it happens whether the hedged read is enabled or not.
      This impacts our 99%tile response time.

      This happens when the block token is expired and the root cause is the wrong handling of the InvalidToken exception in sasl handshake in SaslDataTransferServer.
      I propose to add a new response code for DataTransferEncryptorStatus to request the client to update the block token like DataTransferProtos does.
      The test code and patch is available in https://github.com/apache/hadoop/pull/3677

      We could reproduce this issue by the following test code in 2.8.5 branch and trunk as I tested

      // HDFS is configured as secure cluster
      try (FileSystem fs = newFileSystem();
           FSDataInputStream in = fs.open(PATH)) {
          waitBlockTokenExpired(in);
          in.read(0, bytes, 0, bytes.length)
      }
      
      private void waitBlockTokenExpired(FSDataInputStream in1) throws Exception {
          DFSInputStream innerStream = (DFSInputStream) in1.getWrappedStream();
          for (LocatedBlock block : innerStream.getAllBlocks()) {
              while (!SecurityTestUtil.isBlockTokenExpired(block.getBlockToken())) {
                  Thread.sleep(100);
              }
          }
      }
      

      Here is the log we got, we added a custom log before and after the block token refresh:
      https://github.com/bitterfox/hadoop/commit/173a9f876f2264b76af01d658f624197936fd79c

      2021-11-16 09:40:20,330 WARN  [hedgedRead-247] impl.BlockReaderFactory: I/O error constructing remote block reader.
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      2021-11-16 09:40:20,331 WARN  [hedgedRead-247] hdfs.DFSClient: Connection failure: Failed to connect to /10.10.10.1:12345 for file /hbase/data/default/test_table/<encoded-region-name>/o/<store-file> for block BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924:java.io.IOException: DIGEST-MD5: IO error acquiring password
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      2021-11-16 09:40:20,332 WARN  [hedgedRead-247] impl.BlockReaderFactory: I/O error constructing remote block reader.
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      2021-11-16 09:40:20,332 WARN  [hedgedRead-247] hdfs.DFSClient: Connection failure: Failed to connect to /10.10.10.2:12345 for file /hbase/data/default/test_table/<encoded-region-name>/o/<store-file> for block BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924:java.io.IOException: DIGEST-MD5: IO error acquiring password
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      2021-11-16 09:40:20,333 WARN  [hedgedRead-247] impl.BlockReaderFactory: I/O error constructing remote block reader.
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      2021-11-16 09:40:20,334 WARN  [hedgedRead-247] hdfs.DFSClient: Connection failure: Failed to connect to /10.10.10.3:12345 for file /hbase/data/default/test_table/<encoded-region-name>/o/<store-file> for block BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924:java.io.IOException: DIGEST-MD5: IO error acquiring password
      java.io.IOException: DIGEST-MD5: IO error acquiring password
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil.readSaslMessageAndNegotiatedCipherOption(DataTransferSaslUtil.java:420)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.doSaslHandshake(SaslDataTransferClient.java:475)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.getSaslStreams(SaslDataTransferClient.java:389)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.send(SaslDataTransferClient.java:263)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.checkTrustAndSend(SaslDataTransferClient.java:211)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient.peerSend(SaslDataTransferClient.java:160)
              at org.apache.hadoop.hdfs.DFSUtilClient.peerFromSocketAndKey(DFSUtilClient.java:568)
              at org.apache.hadoop.hdfs.DFSClient.newConnectedPeer(DFSClient.java:2880)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.nextTcpPeer(BlockReaderFactory.java:815)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.getRemoteBlockReaderFromTcp(BlockReaderFactory.java:740)
              at org.apache.hadoop.hdfs.client.impl.BlockReaderFactory.build(BlockReaderFactory.java:385)
              at org.apache.hadoop.hdfs.DFSInputStream.getBlockReader(DFSInputStream.java:697)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1237)
              at org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode(DFSInputStream.java:1205)
              at org.apache.hadoop.hdfs.DFSInputStream.access$000(DFSInputStream.java:98)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1189)
              at org.apache.hadoop.hdfs.DFSInputStream$2.call(DFSInputStream.java:1181)
              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.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      
      2021-11-16 09:40:20,334 WARN  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: No live nodes contain block BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924 after checking nodes = [DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]], ignoredNodes = [DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]]
      2021-11-16 09:40:20,334 INFO  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: Could not obtain BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924 from any node:  No live nodes contain current block Block locations: DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK] Dead nodes:  DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK] Ignored nodes:  DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]. Will get new block locations from namenode and retry...
      2021-11-16 09:40:20,334 WARN  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: DFS chooseDataNode: got # 1 IOException, will wait for 523.0985342660828 msec.
      2021-11-16 09:40:20,860 WARN  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: No live nodes contain block BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924 after checking nodes = [DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]], ignoredNodes = [DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK], DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]]
      2021-11-16 09:40:20,860 INFO  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: Could not obtain BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924 from any node:  No live nodes contain current block Block locations: DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK] Dead nodes:  Ignored nodes:  DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]. Will get new block locations from namenode and retry...
      2021-11-16 09:40:20,860 WARN  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: DFS chooseDataNode: got # 2 IOException, will wait for 8025.758935908773 msec.
      2021-11-16 09:40:28,887 INFO  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: Could not obtain BP-123456789-10.20.20.1-1629777195910:blk_9876543212_1357924 from any node:  No live nodes contain current block Block locations: DatanodeInfoWithStorage[10.10.10.2:12345,DS-*****,
      DISK] DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK] Dead nodes:  Ignored nodes:  DatanodeInfoWithStorage[10.10.10.1:12345,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.2:1146
      2,DS-*****,DISK] DatanodeInfoWithStorage[10.10.10.3:12345,DS-*****,DISK]. Will get new block locations from namenode and retry...
      2021-11-16 09:40:28,887 WARN  [RpcServer.default.RWQ.Fifo.read.handler=363,queue=28,port=10001] hdfs.DFSClient: DFS chooseDataNode: got # 3 IOException, will wait for 7995.183785064122 msec.
      2021-11-16 09:40:59,922 WARN  [RpcServer.default.RWQ.Fifo.read.handler=371,queue=36,port=10001] ipc.RpcServer: (responseTooSlow): {"call":"Multi(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$MultiRequest)","multi.gets":3,"starttimems":"1637023220329","responsesize":"64393","method":"Multi","param":"region\u003d test_table,***,1631095286710.<encoded-region-name>., for 3 action(s) and 1st row keyTRUNCATED","processingtimems":39592,"client":"10.30.30.1:56789","queuetimems":0,"multi.servicecalls":0,"class":"HRegionServer","multi.mutations":0}
      

      As you can see, you see the IOException and then all datanodes are considered dead nodes.
      Also, you couldn't see the block token refresh occurring.

      So the logic of refresh block token isn't performed for some cases and all datanodes are marked as dead and then chooseDataNode and refetchLocations is triggered with the sleep.
      refetchLocations sleeps up to `dfsClient.getConf().getTimeWindow()` default is 3 second for first failure.
      https://github.com/apache/hadoop/blob/91af256a5b44925e5dfdf333293251a19685ba2a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L982-L1046

      That's why we see slows 0~3 secs for hedged read disabled.

      refetchLocations clears dead nodes, but ignored node that managed in hedgedFetchBlockByteRange are not cleared, so hedgedFetchBlockByteRange tries refetchLocations many times up to `dfsClient.getConf().getMaxBlockAcquireFailures()`
      (sleep in refetchLocations is computed by `timeWindow * failure + timeWindow * (failure + 1) * nextDouble()` and that's why we see several minutes response slow when hedged read is enabled)
      After these retries, BlockMissingException is thrown.
      https://github.com/apache/hadoop/blob/91af256a5b44925e5dfdf333293251a19685ba2a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java#L1343-L1386

      We dig into the IOException stacktrace and we found sasl handshake returns an error.
      We added the log in SaslDataTransferServer side:
      https://github.com/bitterfox/hadoop/tree/saslhandshake-error-log

      and then we got the following DN error:

      2021-11-16 16:11:06,480 ERROR org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer: Generic sasl error for client /10.10.10.4:45678
      javax.security.sasl.SaslException: DIGEST-MD5: IO error acquiring password [Caused by org.apache.hadoop.security.token.SecretManager$InvalidToken: Block token with block_token_identifier (expiryDate=1637046306844, keyId=<keyid>, userId=hbase, blockPoolId=BP-123456789-10.20.20.1-1629777195910, blockId=<blockid>, access modes=[READ]) is expired.]
              at com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:598)
              at com.sun.security.sasl.digest.DigestMD5Server.evaluateResponse(DigestMD5Server.java:244)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslParticipant.evaluateChallengeOrResponse(SaslParticipant.java:115)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.doSaslHandshake(SaslDataTransferServer.java:376)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.getSaslStreams(SaslDataTransferServer.java:300)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.receive(SaslDataTransferServer.java:127)
              at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:231)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.hadoop.security.token.SecretManager$InvalidToken: Block token with block_token_identifier (expiryDate=1637046306844, keyId=<keyid>, userId=hbase, blockPoolId=BP-123456789-10.20.20.1-1629777195910, blockId=<blockid>, access modes=[READ]) is expired.
              at org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager.retrievePassword(BlockTokenSecretManager.java:377)
              at org.apache.hadoop.hdfs.security.token.block.BlockPoolTokenSecretManager.retrievePassword(BlockPoolTokenSecretManager.java:80)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.buildServerPassword(SaslDataTransferServer.java:318)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer.access$100(SaslDataTransferServer.java:73)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$2.apply(SaslDataTransferServer.java:297)
              at org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferServer$SaslServerCallbackHandler.handle(SaslDataTransferServer.java:241)
              at com.sun.security.sasl.digest.DigestMD5Server.validateClientResponse(DigestMD5Server.java:589)
              ... 7 more
      

      As you can see the expired token is used and retrievePassword used for sasl throws InvalidToken exception.
      retrievePassword: https://github.com/apache/hadoop/blob/91af256a5b44925e5dfdf333293251a19685ba2a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/block/BlockTokenSecretManager.java#L501-L506

      So if a connection is established newly after the block token is expired, this issue happens.

      Attachments

        1. Screenshot from 2021-11-18 13-31-35.png
          427 kB
          Shinya Yoshida
        2. Screenshot from 2021-11-18 12-11-34.png
          570 kB
          Shinya Yoshida
        3. Screenshot from 2021-11-18 12-14-29.png
          107 kB
          Shinya Yoshida

        Issue Links

          Activity

            People

              lineyshinya Shinya Yoshida
              lineyshinya Shinya Yoshida
              Votes:
              2 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - Not Specified
                  Not Specified
                  Remaining:
                  Remaining Estimate - 0h
                  0h
                  Logged:
                  Time Spent - 5h 40m
                  5h 40m