[ 
https://issues.apache.org/jira/browse/HDFS-16456?focusedWorklogId=769424&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-769424
 ]

ASF GitHub Bot logged work on HDFS-16456:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/May/22 05:25
            Start Date: 12/May/22 05:25
    Worklog Time Spent: 10m 
      Work Description: jojochuang opened a new pull request, #4304:
URL: https://github.com/apache/hadoop/pull/4304

   
   
   (cherry picked from commit cee8c62498f55794f911ce62edfd4be9e88a7361)
   
    Conflicts:
        
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java
   
   Change-Id: I5da7693d6176315f126f1b5107de2bb100ee9778
   
   <!--
     Thanks for sending a pull request!
       1. If this is your first time, please read our contributor guidelines: 
https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
       2. Make sure your PR title starts with JIRA issue id, e.g., 
'HADOOP-17799. Your PR title ...'.
   -->
   
   ### Description of PR
   (HDFS-16456 was merged in trunk. It has conflict in branch-3.3 because of 
HDFS-15263. It looks like HDFS-15263 could potentially change the behavior and 
surprise users, so decided to backport without HDFS-15263)
   
   ### How was this patch tested?
   
   
   ### For code changes:
   
   - [ ] Does the title or this PR starts with the corresponding JIRA issue id 
(e.g. 'HADOOP-17799. Your PR title ...')?
   - [ ] Object storage: have the integration tests been executed and the 
endpoint declared according to the connector-specific documentation?
   - [ ] If adding new dependencies to the code, are these dependencies 
licensed in a way that is compatible for inclusion under [ASF 
2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`, 
`NOTICE-binary` files?
   
   




Issue Time Tracking
-------------------

    Worklog Id:     (was: 769424)
    Time Spent: 2h 20m  (was: 2h 10m)

> EC: Decommission a rack with only on dn will fail when the rack number is 
> equal with replication
> ------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-16456
>                 URL: https://issues.apache.org/jira/browse/HDFS-16456
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: ec, namenode
>    Affects Versions: 3.4.0
>            Reporter: caozhiqiang
>            Assignee: caozhiqiang
>            Priority: Critical
>              Labels: pull-request-available
>             Fix For: 3.4.0
>
>         Attachments: HDFS-16456.001.patch, HDFS-16456.002.patch, 
> HDFS-16456.003.patch, HDFS-16456.004.patch, HDFS-16456.005.patch, 
> HDFS-16456.006.patch, HDFS-16456.007.patch, HDFS-16456.008.patch, 
> HDFS-16456.009.patch, HDFS-16456.010.patch
>
>          Time Spent: 2h 20m
>  Remaining Estimate: 0h
>
> In below scenario, decommission will fail by TOO_MANY_NODES_ON_RACK reason:
>  # Enable EC policy, such as RS-6-3-1024k.
>  # The rack number in this cluster is equal with or less than the replication 
> number(9)
>  # A rack only has one DN, and decommission this DN.
> The root cause is in 
> BlockPlacementPolicyRackFaultTolerant::getMaxNodesPerRack() function, it will 
> give a limit parameter maxNodesPerRack for choose targets. In this scenario, 
> the maxNodesPerRack is 1, which means each rack can only be chosen one 
> datanode.
> {code:java}
>   protected int[] getMaxNodesPerRack(int numOfChosen, int numOfReplicas) {
>    ...
>     // If more replicas than racks, evenly spread the replicas.
>     // This calculation rounds up.
>     int maxNodesPerRack = (totalNumOfReplicas - 1) / numOfRacks + 1;
>     return new int[] {numOfReplicas, maxNodesPerRack};
>   } {code}
> int maxNodesPerRack = (totalNumOfReplicas - 1) / numOfRacks + 1;
> here will be called, where totalNumOfReplicas=9 and  numOfRacks=9  
> When we decommission one dn which is only one node in its rack, the 
> chooseOnce() in BlockPlacementPolicyRackFaultTolerant::chooseTargetInOrder() 
> will throw NotEnoughReplicasException, but the exception will not be caught 
> and fail to fallback to chooseEvenlyFromRemainingRacks() function.
> When decommission, after choose targets, verifyBlockPlacement() function will 
> return the total rack number contains the invalid rack, and 
> BlockPlacementStatusDefault::isPlacementPolicySatisfied() will return false 
> and it will also cause decommission fail.
> {code:java}
>   public BlockPlacementStatus verifyBlockPlacement(DatanodeInfo[] locs,
>       int numberOfReplicas) {
>     if (locs == null)
>       locs = DatanodeDescriptor.EMPTY_ARRAY;
>     if (!clusterMap.hasClusterEverBeenMultiRack()) {
>       // only one rack
>       return new BlockPlacementStatusDefault(1, 1, 1);
>     }
>     // Count locations on different racks.
>     Set<String> racks = new HashSet<>();
>     for (DatanodeInfo dn : locs) {
>       racks.add(dn.getNetworkLocation());
>     }
>     return new BlockPlacementStatusDefault(racks.size(), numberOfReplicas,
>         clusterMap.getNumOfRacks());
>   } {code}
> {code:java}
>   public boolean isPlacementPolicySatisfied() {
>     return requiredRacks <= currentRacks || currentRacks >= totalRacks;
>   }{code}
> According to the above description, we should make the below modify to fix it:
>  # In startDecommission() or stopDecommission(), we should also change the 
> numOfRacks in class NetworkTopology. Or choose targets may fail for the 
> maxNodesPerRack is too small. And even choose targets success, 
> isPlacementPolicySatisfied will also return false cause decommission fail.
>  # In BlockPlacementPolicyRackFaultTolerant::chooseTargetInOrder(), the first 
> chooseOnce() function should also be put in try..catch..., or it will not 
> fallback to call chooseEvenlyFromRemainingRacks() when throw exception.
>  # In verifyBlockPlacement, we need to remove invalid racks from total 
> numOfRacks, or isPlacementPolicySatisfied() will return false and cause fail 
> to reconstruct data.
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

---------------------------------------------------------------------
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