[
https://issues.apache.org/jira/browse/HDFS-15237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17065461#comment-17065461
]
zhengchenyu edited comment on HDFS-15237 at 3/24/20, 9:42 AM:
--------------------------------------------------------------
Here comment some strange phenomenon. Why this error happen? Because I found
some wrong internal block distribution like this:
{code:java}
0. BP-1936287042-10.200.128.33-1573194961291:blk_-9223372036797335472_3783205
len=247453749 Live_repl=8
[blk_-9223372036797335472:DatanodeInfoWithStorage[10.200.128.43:9866,DS-2ddde0b8-6a84-4d06-8a40-d4ae5691e81c,DISK],
blk_-9223372036797335471:DatanodeInfoWithStorage[10.200.128.41:9866,DS-a4fc5486-6c45-481e-84e7-9393eeaf1313,DISK],
blk_-9223372036797335470:DatanodeInfoWithStorage[10.200.128.50:9866,DS-fc0632c6-8916-42d8-8219-57b022bb2786,DISK],
blk_-9223372036797335469:DatanodeInfoWithStorage[10.200.128.54:9866,DS-1b6cb52a-f55a-4ef8-beaf-a5d7b7fe93aa,DISK],
blk_-9223372036797335467:DatanodeInfoWithStorage[10.200.128.52:9866,DS-fc6e00dd-ca5a-4580-9403-aeb6906da81a,DISK],
blk_-9223372036797335466:DatanodeInfoWithStorage[10.200.128.53:9866,DS-2c926a3b-64c0-441b-abe2-188e79918abe,DISK],
blk_-9223372036797335465:DatanodeInfoWithStorage[10.200.128.40:9866,DS-65ac4407-9d33-4c59-8f72-dd1d80d26d9f,DISK],
blk_-9223372036797335464:DatanodeInfoWithStorage[10.200.128.44:9866,DS-3725af76-fe86-4f97-9740-d77bfa339b3f,DISK],
blk_-9223372036797335470:DatanodeInfoWithStorage[10.200.128.45:9866,DS-250fd4cf-705f-4cb5-bc3a-c7a105247e35,DISK]]
{code}
this is the result of hdfs fsck. Your can see this block group has 9 internal
block, but no blk_-9223372036797335468, two repeated blk_-9223372036797335470.
this block is too old so that the log is missing, so I don't know the reason,
and can't reproduction this error now.
was (Author: zhengchenyu):
Here comment some strange phenomenon. Why this error happen? Because I found
some wrong internal block distribution like this:
{code}
0. BP-1936287042-10.200.128.33-1573194961291:blk_-9223372036797335472_3783205
len=247453749 Live_repl=8
[blk_-9223372036797335472:DatanodeInfoWithStorage[10.200.128.43:9866,DS-2ddde0b8-6a84-4d06-8a40-d4ae5691e81c,DISK],
blk_-9223372036797335471:DatanodeInfoWithStorage[10.200.128.41:9866,DS-a4fc5486-6c45-481e-84e7-9393eeaf1313,DISK],
blk_-9223372036797335470:DatanodeInfoWithStorage[10.200.128.50:9866,DS-fc0632c6-8916-42d8-8219-57b022bb2786,DISK],
blk_-9223372036797335469:DatanodeInfoWithStorage[10.200.128.54:9866,DS-1b6cb52a-f55a-4ef8-beaf-a5d7b7fe93aa,DISK],
blk_-9223372036797335467:DatanodeInfoWithStorage[10.200.128.52:9866,DS-fc6e00dd-ca5a-4580-9403-aeb6906da81a,DISK],
blk_-9223372036797335466:DatanodeInfoWithStorage[10.200.128.53:9866,DS-2c926a3b-64c0-441b-abe2-188e79918abe,DISK],
blk_-9223372036797335465:DatanodeInfoWithStorage[10.200.128.40:9866,DS-65ac4407-9d33-4c59-8f72-dd1d80d26d9f,DISK],
blk_-9223372036797335464:DatanodeInfoWithStorage[10.200.128.44:9866,DS-3725af76-fe86-4f97-9740-d77bfa339b3f,DISK],
blk_-9223372036797335470:DatanodeInfoWithStorage[10.200.128.45:9866,DS-250fd4cf-705f-4cb5-bc3a-c7a105247e35,DISK]]
{code}
this is the result of hdfs fsck. Your can see this block group has 9 internal
block, but no blk_-9223372036797335468, two repeated blk_-9223372036797335480.
this block is too old so that the log is missing, so I don't know the reason,
and can't reproduction this error now.
> Get checksum of EC file failed, when some block is missing or corrupt
> ---------------------------------------------------------------------
>
> Key: HDFS-15237
> URL: https://issues.apache.org/jira/browse/HDFS-15237
> Project: Hadoop HDFS
> Issue Type: Bug
> Components: ec, hdfs
> Affects Versions: 3.2.1
> Reporter: zhengchenyu
> Priority: Major
> Fix For: 3.2.2
>
>
> When we distcp from an ec directory to another one, I found some error like
> this.
> {code}
> 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)
> {code}
> And Then I found some error in datanode like this
> {code}
> 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)
> {code}
> 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.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]