[ 
https://issues.apache.org/jira/browse/HDFS-14768?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16925298#comment-16925298
 ] 

guojh edited comment on HDFS-14768 at 9/9/19 2:14 AM:
------------------------------------------------------

[~surendrasingh] Thanks for you replay. you need run the code below, and you 
need check the block index 6 that recontruct on local path like 
'./hadoop/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs/*/current/BP-xxxxxxx/current/finalized/*/*/'
{code:java}
// code placeholder
public void testFileDecommission() throws Exception { LOG.info("Starting test 
testFileDecommission"); final Path ecFile = new Path(ecDir, 
"testFileDecommission"); int writeBytes = cellSize * dataBlocks; 
writeStripedFile(dfs, ecFile, writeBytes); Assert.assertEquals(0, 
bm.numOfUnderReplicatedBlocks()); FileChecksum fileChecksum1 = 
dfs.getFileChecksum(ecFile, writeBytes); LocatedBlocks locatedBlocks = 
StripedFileTestUtil.getLocatedBlocks(ecFile, dfs); LocatedBlock lb = 
dfs.getClient().getLocatedBlocks(ecFile.toString(), 0) .get(0); DatanodeInfo[] 
dnLocs = lb.getLocations(); LocatedStripedBlock lastBlock = 
(LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); DatanodeInfo[] 
storageInfos = lastBlock.getLocations();  DatanodeDescriptor datanodeDescriptor 
= cluster.getNameNode().getNamesystem() 
.getBlockManager().getDatanodeManager().getDatanode(storageInfos[6].getDatanodeUuid());
 for (int i = 0; i < 100; i++) { 
datanodeDescriptor.incrementPendingReplicationWithoutTargets(); } 
assertEquals(dataBlocks + parityBlocks, dnLocs.length); int[] decommNodeIndex = 
{3, 4}; final List<DatanodeInfo> decommisionNodes = new 
ArrayList<DatanodeInfo();
decommisionNodes.add(dnLocs[decommNodeIndex[0]]); 
decommisionNodes.add(dnLocs[decommNodeIndex[1]]); decommissionNode(0, 
decommisionNodes, AdminStates.DECOMMISSIONED); 
assertEquals(decommisionNodes.size(), fsn.getNumDecomLiveDataNodes()); 
assertNull(checkFile(dfs, ecFile, 9, decommisionNodes, numDNs)); DFSClient 
client = getDfsClient(cluster.getNameNode(0), conf); assertEquals("All 
datanodes must be alive", numDNs, 
client.datanodeReport(DatanodeReportType.LIVE).length); FileChecksum 
fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes); 
Assert.assertTrue("Checksum mismatches!", fileChecksum1.equals(fileChecksum2)); 
StripedFileTestUtil.checkData(dfs, ecFile, writeBytes, decommisionNodes, null, 
blockGroupSize); }
{code}
 


was (Author: gjhkael):
[~surendrasingh] Thanks for you replay. you need run the code below, and you 
need check the block index 6 that recontruct on local path like 
'./hadoop/hadoop-hdfs-project/hadoop-hdfs/target/test/data/dfs/*/current/BP-xxxxxxx/current/finalized/*/*/'
{code:java}
// code placeholder

{code}
 

> In some cases, erasure blocks are corruption  when they are reconstruct.
> ------------------------------------------------------------------------
>
>                 Key: HDFS-14768
>                 URL: https://issues.apache.org/jira/browse/HDFS-14768
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: datanode, erasure-coding, hdfs, namenode
>    Affects Versions: 3.0.2
>            Reporter: guojh
>            Assignee: guojh
>            Priority: Major
>              Labels: patch
>             Fix For: 3.3.0
>
>         Attachments: HDFS-14768.000.patch
>
>
> Policy is RS-6-3-1024K, version is hadoop 3.0.2;
> We suppose a file's block Index is [0,1,2,3,4,5,6,7,8], And decommission 
> index[3,4], increase the index 6 datanode's
> pendingReplicationWithoutTargets  that make it large than 
> replicationStreamsHardLimit(we set 14). Then, After the method 
> chooseSourceDatanodes of BlockMananger, the liveBlockIndices is 
> [0,1,2,3,4,5,7,8], Block Counter is, Live:7, Decommission:2. 
> In method scheduleReconstruction of BlockManager, the additionalReplRequired 
> is 9 - 7 = 2. After Namenode choose two target Datanode, will assign a 
> erasureCode task to target datanode.
> When datanode get the task will build  targetIndices from liveBlockIndices 
> and target length. the code is blow.
> {code:java}
> // code placeholder
> targetIndices = new short[targets.length];
> private void initTargetIndices() { 
>   BitSet bitset = reconstructor.getLiveBitSet();
>   int m = 0; hasValidTargets = false; 
>   for (int i = 0; i < dataBlkNum + parityBlkNum; i++) {  
>     if (!bitset.get) {    
>       if (reconstructor.getBlockLen > 0) {
>        if (m < targets.length) {
>          targetIndices[m++] = (short)i;
>          hasValidTargets = true;
>         }
>       }
>     }
>  }
> {code}
> targetIndices[0]=6, and targetIndices[1] is aways 0 from initial value.
> The StripedReader is  aways create reader from first 6 index block, and is 
> [0,1,2,3,4,5]
> Use the index [0,1,2,3,4,5] to build target index[6,0] will trigger the isal 
> bug. the block index6's data is corruption(all data is zero).
> I write a unit test can stabilize repreduce.
> {code:java}
> // code placeholder
> public void testFileDecommission() throws Exception {
>   LOG.info("Starting test testFileDecommission");
>   final Path ecFile = new Path(ecDir, "testFileDecommission");
>   int writeBytes = cellSize * dataBlocks;
>   writeStripedFile(dfs, ecFile, writeBytes);
>   Assert.assertEquals(0, bm.numOfUnderReplicatedBlocks());
>   FileChecksum fileChecksum1 = dfs.getFileChecksum(ecFile, writeBytes);
>   LocatedBlocks locatedBlocks =
>       StripedFileTestUtil.getLocatedBlocks(ecFile, dfs);
>   LocatedBlock lb = dfs.getClient().getLocatedBlocks(ecFile.toString(), 0)
>       .get(0);
>   DatanodeInfo[] dnLocs = lb.getLocations();
>   LocatedStripedBlock lastBlock =
>       (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock();
>   DatanodeInfo[] storageInfos = lastBlock.getLocations();
>   // 
>   DatanodeDescriptor datanodeDescriptor = 
> cluster.getNameNode().getNamesystem()
>       
> .getBlockManager().getDatanodeManager().getDatanode(storageInfos[6].getDatanodeUuid());
>   for (int i = 0; i < 100; i++) {
>     datanodeDescriptor.incrementPendingReplicationWithoutTargets();
>   }
>   assertEquals(dataBlocks + parityBlocks, dnLocs.length);
>   int[] decommNodeIndex = {3, 4};
>   final List<DatanodeInfo> decommisionNodes = new ArrayList<DatanodeInfo>();
>   // add the node which will be decommissioning
>   decommisionNodes.add(dnLocs[decommNodeIndex[0]]);
>   decommisionNodes.add(dnLocs[decommNodeIndex[1]]);
>   decommissionNode(0, decommisionNodes, AdminStates.DECOMMISSIONED);
>   assertEquals(decommisionNodes.size(), fsn.getNumDecomLiveDataNodes());
>   //assertNull(checkFile(dfs, ecFile, 9, decommisionNodes, numDNs));
>   // Ensure decommissioned datanode is not automatically shutdown
>   DFSClient client = getDfsClient(cluster.getNameNode(0), conf);
>   assertEquals("All datanodes must be alive", numDNs,
>       client.datanodeReport(DatanodeReportType.LIVE).length);
>   FileChecksum fileChecksum2 = dfs.getFileChecksum(ecFile, writeBytes);
>   Assert.assertTrue("Checksum mismatches!",
>       fileChecksum1.equals(fileChecksum2));
>   StripedFileTestUtil.checkData(dfs, ecFile, writeBytes, decommisionNodes,
>       null, blockGroupSize);
> }
> {code}
>  



--
This message was sent by Atlassian Jira
(v8.3.2#803003)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to