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

Get checksum of EC file failed, when some block is missing or corrupt

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Duplicate
    • 3.2.1
    • None
    • ec, hdfs

    Description

      When we distcp from an ec directory to another one, I found some error like this.

      
      2020-03-20 20:18:21,366 WARN [main] org.apache.hadoop.hdfs.FileChecksumHelper: src=/EC/6-3/****/000325_0, datanodes[6]=DatanodeInfoWithStorage[10.200.128.40:9866,DS-65ac4407-9d33-4c59-8f72-dd1d80d26d9f,DISK]2020-03-20 20:18:21,366 WARN [main] org.apache.hadoop.hdfs.FileChecksumHelper: src=/EC/6-3/****/000325_0, datanodes[6]=DatanodeInfoWithStorage[10.200.128.40:9866,DS-65ac4407-9d33-4c59-8f72-dd1d80d26d9f,DISK]java.io.EOFException: Unexpected EOF while trying to read response from server at org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed(PBHelperClient.java:550) at org.apache.hadoop.hdfs.FileChecksumHelper$StripedFileNonStripedChecksumComputer.tryDatanode(FileChecksumHelper.java:709) at org.apache.hadoop.hdfs.FileChecksumHelper$StripedFileNonStripedChecksumComputer.checksumBlockGroup(FileChecksumHelper.java:664) at org.apache.hadoop.hdfs.FileChecksumHelper$StripedFileNonStripedChecksumComputer.checksumBlocks(FileChecksumHelper.java:638) at org.apache.hadoop.hdfs.FileChecksumHelper$FileChecksumComputer.compute(FileChecksumHelper.java:252) at org.apache.hadoop.hdfs.DFSClient.getFileChecksumInternal(DFSClient.java:1790) at org.apache.hadoop.hdfs.DFSClient.getFileChecksumWithCombineMode(DFSClient.java:1810) at org.apache.hadoop.hdfs.DistributedFileSystem$33.doCall(DistributedFileSystem.java:1691) at org.apache.hadoop.hdfs.DistributedFileSystem$33.doCall(DistributedFileSystem.java:1688) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.getFileChecksum(DistributedFileSystem.java:1700) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doCopy(RetriableFileCopyCommand.java:138) at org.apache.hadoop.tools.mapred.RetriableFileCopyCommand.doExecute(RetriableFileCopyCommand.java:115) at org.apache.hadoop.tools.util.RetriableCommand.execute(RetriableCommand.java:87) at org.apache.hadoop.tools.mapred.CopyMapper.copyFileWithRetry(CopyMapper.java:259) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:220) at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:48) at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:146) at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:799) at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347) at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1730) at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
      
      

      And Then I found some error in datanode like this

      
      2020-03-20 20:54:16,573 INFO org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient: SASL encryption trust check: localHostTrusted = false, remoteHostTrusted = false
      2020-03-20 20:54:16,577 ERROR org.apache.hadoop.hdfs.server.datanode.DataNode: bd-hadoop-128050.zeus.lianjia.com:9866:DataXceiver error processing BLOCK_GROUP_CHECKSUM operation src: /10.201.1.38:33264 dst: /10.200.128.50:9866
      java.lang.UnsupportedOperationException
       at java.nio.ByteBuffer.array(ByteBuffer.java:994)
       at org.apache.hadoop.hdfs.server.datanode.erasurecode.StripedBlockChecksumReconstructor.reconstruct(StripedBlockChecksumReconstructor.java:90)
       at org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper$BlockGroupNonStripedChecksumComputer.recalculateChecksum(BlockChecksumHelper.java:711)
       at org.apache.hadoop.hdfs.server.datanode.BlockChecksumHelper$BlockGroupNonStripedChecksumComputer.compute(BlockChecksumHelper.java:489)
       at org.apache.hadoop.hdfs.server.datanode.DataXceiver.blockGroupChecksum(DataXceiver.java:1047)
       at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opStripedBlockChecksum(Receiver.java:327)
       at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:119)
       at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:292)
       at java.lang.Thread.run(Thread.java:748)
      
      

      The reason is that: When some block is missing or corrupt, datanode will trigger to call recalculateChecksum. But if StripedBlockChecksumReconstructor.targetBuffer is DirectByteBuffer, we couldn't use DirectByteBuffer.array(), so throw the exception. Then we could't get checksum.

      Attachments

        1. HDFS-15237.001.patch
          2 kB
          Chenyu Zheng

        Issue Links

          Activity

            People

              Unassigned Unassigned
              zhengchenyu Chenyu Zheng
              Votes:
              0 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 - 1h 10m
                  1h 10m