You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-issues@hadoop.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2022/06/02 19:02:00 UTC

[jira] [Work logged] (HDFS-16456) EC: Decommission a rack with only on dn will fail when the rack number is equal with replication

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

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

                Author: ASF GitHub Bot
            Created on: 02/Jun/22 19:01
            Start Date: 02/Jun/22 19:01
    Worklog Time Spent: 10m 
      Work Description: jojochuang commented on PR #4358:
URL: https://github.com/apache/hadoop/pull/4358#issuecomment-1145210866

   The failed tests do not fail in my local tree. I'll merge the change later.




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

    Worklog Id:     (was: 777787)
    Time Spent: 3.5h  (was: 3h 20m)

> 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, 3.3.4
>
>         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: 3.5h
>  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-unsubscribe@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org