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

Exception when client reports bad checksum to NN

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Critical
    • Resolution: Fixed
    • Affects Version/s: 2.0.0-alpha, 0.23.5
    • Fix Version/s: None
    • Component/s: hdfs-client, namenode
    • Labels:
      None

      Description

      We see the following exception in our logs on a cluster:

      2012-08-27 16:34:30,400 INFO org.apache.hadoop.hdfs.StateChange: *DIR* NameNode.reportBadBlocks
      2012-08-27 16:34:30,400 ERROR org.apache.hadoop.security.UserGroupInformation: PriviledgedActionException as:hdfs (auth:SIMPLE) cause:java.io.IOException: Cannot mark blk_8285012733733669474_140475196{blockUCState=UNDER_CONSTRUCTION, primaryNodeIndex=-1, replicas=[ReplicaUnderConstruction[172.29.97.219:50010|RBW]]}(same as stored) as corrupt because datanode :0 does not exist
      2012-08-27 16:34:30,400 INFO org.apache.hadoop.ipc.Server: IPC Server handler 46 on 8020, call org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol.reportBadBlocks from 172.29.97.219:43805: error: java.io.IOException: Cannot mark blk_8285012733733669474_140475196{blockUCState=UNDER_CONSTRUCTION, primaryNodeIndex=-1, replicas=[ReplicaUnderConstruction[172.29.97.219:50010|RBW]]}(same as stored) as corrupt because datanode :0 does not exist
      java.io.IOException: Cannot mark blk_8285012733733669474_140475196{blockUCState=UNDER_CONSTRUCTION, primaryNodeIndex=-1, replicas=[ReplicaUnderConstruction[172.29.97.219:50010|RBW]]}(same as stored) as corrupt because datanode :0 does not exist
              at org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.markBlockAsCorrupt(BlockManager.java:1001)
              at org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.findAndMarkBlockAsCorrupt(BlockManager.java:994)
              at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.reportBadBlocks(FSNamesystem.java:4736)
              at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.reportBadBlocks(NameNodeRpcServer.java:537)
              at org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolServerSideTranslatorPB.reportBadBlocks(DatanodeProtocolServerSideTranslatorPB.java:242)
              at org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos$DatanodeProtocolService$2.callBlockingMethod(DatanodeProtocolProtos.java:20032)
              at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:453)
      

        Issue Links

          Activity

          Hide
          tlipcon Todd Lipcon added a comment -

          The bug seems to be that the datanode doesn't report the right remote DN when it detects a checksum error when receiving a block. Here are the DN side logs:

          2012-08-27 16:34:30,396 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Checksum error in block BP-1507505631-172.29.97.196-1337120439433:blk_8285012733733669474_140475196 from /172.29.97.219:52544
          org.apache.hadoop.fs.ChecksumException: Checksum error: DFSClient_NONMAPREDUCE_334070927_1 at 44032 exp: -983390667 got: 557443094
                  at org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:335)
                  at org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:266)
                  at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.verifyChunks(BlockReceiver.java:377)
                  at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:496)
                  at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:635)
                  at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:506)
                  at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:98)
                  at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:66)
                  at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:219)
                  at java.lang.Thread.run(Thread.java:662)
          2012-08-27 16:34:30,396 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: report corrupt block BP-1507505631-172.29.97.196-1337120439433:blk_8285012733733669474_140475196 from datanode :0 to namenode
          
          Show
          tlipcon Todd Lipcon added a comment - The bug seems to be that the datanode doesn't report the right remote DN when it detects a checksum error when receiving a block. Here are the DN side logs: 2012-08-27 16:34:30,396 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: Checksum error in block BP-1507505631-172.29.97.196-1337120439433:blk_8285012733733669474_140475196 from /172.29.97.219:52544 org.apache.hadoop.fs.ChecksumException: Checksum error: DFSClient_NONMAPREDUCE_334070927_1 at 44032 exp: -983390667 got: 557443094 at org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:335) at org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:266) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.verifyChunks(BlockReceiver.java:377) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:496) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:635) at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:506) at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:98) at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:66) at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:219) at java.lang. Thread .run( Thread .java:662) 2012-08-27 16:34:30,396 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: report corrupt block BP-1507505631-172.29.97.196-1337120439433:blk_8285012733733669474_140475196 from datanode :0 to namenode
          Hide
          kihwal Kihwal Lee added a comment -

          I've seen this happening. This is worse than it looks. In 3-repl/2-min_repl case, the last datanode in the pipeline does not report anything and the pipeline is recreated with the remaining two nodes. The problem is the two nodes may have already written the corrupt data to disk. The reconstructed pipeline will be used and the block will complete. Once the block is done, NN will schedule replication, which will fail the two sources one by one, causing the block to be "missing".

          Looking at the code, the source DatanodeId used in corruption reporting is propagated from the client. But when DFSClient calls writeBlock(), it passes null as srcNode, so no one in the pipline has valid srcNode. Maybe NN should check whether the block is under construction and the reporter was the last one in the pipeline. In this case, all copies of the blocks should be marked as corrupt.

          In addition to this, the last one in the pipeline should synchronously return an appropriate failure, instead of simply going away.

          Show
          kihwal Kihwal Lee added a comment - I've seen this happening. This is worse than it looks. In 3-repl/2-min_repl case, the last datanode in the pipeline does not report anything and the pipeline is recreated with the remaining two nodes. The problem is the two nodes may have already written the corrupt data to disk. The reconstructed pipeline will be used and the block will complete. Once the block is done, NN will schedule replication, which will fail the two sources one by one, causing the block to be "missing". Looking at the code, the source DatanodeId used in corruption reporting is propagated from the client. But when DFSClient calls writeBlock(), it passes null as srcNode, so no one in the pipline has valid srcNode. Maybe NN should check whether the block is under construction and the reporter was the last one in the pipeline. In this case, all copies of the blocks should be marked as corrupt. In addition to this, the last one in the pipeline should synchronously return an appropriate failure, instead of simply going away.
          Hide
          tlipcon Todd Lipcon added a comment -

          Hi Kihwal. The issue you're describing also sounds somewhat like HDFS-3875. Have you seen that one? Would appreciate your thoughts there.

          Show
          tlipcon Todd Lipcon added a comment - Hi Kihwal. The issue you're describing also sounds somewhat like HDFS-3875 . Have you seen that one? Would appreciate your thoughts there.
          Hide
          kihwal Kihwal Lee added a comment -

          In branch-1, srcNode is created from the result of getRemoteSocketAddress(), so it contained peer's address instead of null passed down from client. If we do something equivalent, the last one in the pipeline will be able to report the second to the last, but that is not enough to cover all the cases. The last one needs to be included when determining up to which offset the data is sound. If the last DN simply disappears like today, this won't happen.

          Since HDFS-3875 talks about this specific issue, we will continue the discussion there.

          Show
          kihwal Kihwal Lee added a comment - In branch-1, srcNode is created from the result of getRemoteSocketAddress(), so it contained peer's address instead of null passed down from client. If we do something equivalent, the last one in the pipeline will be able to report the second to the last, but that is not enough to cover all the cases. The last one needs to be included when determining up to which offset the data is sound. If the last DN simply disappears like today, this won't happen. Since HDFS-3875 talks about this specific issue, we will continue the discussion there.
          Hide
          kihwal Kihwal Lee added a comment -

          This defect will be fixed as part of HDFS-3875.

          Show
          kihwal Kihwal Lee added a comment - This defect will be fixed as part of HDFS-3875 .

            People

            • Assignee:
              kihwal Kihwal Lee
              Reporter:
              tlipcon Todd Lipcon
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development