You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/06/30 11:02:03 UTC

[GitHub] [ozone] JacksonYao287 opened a new pull request, #3572: HDDS-6960. EC: Implement the Over-replication Handler

JacksonYao287 opened a new pull request, #3572:
URL: https://github.com/apache/ozone/pull/3572

   ## What changes were proposed in this pull request?
   
   Implement the Over-replication Handler
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6960
   
   ## How was this patch tested?
   
   unit test
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] umamaheswararao commented on a diff in pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on code in PR #3572:
URL: https://github.com/apache/ozone/pull/3572#discussion_r912686443


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/AbstractOverReplicationHandler.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * this class holds some common methods that will be shared among
+ * different kinds of Implementation of OverReplicationHandler.
+ * */
+public abstract class AbstractOverReplicationHandler
+    implements UnhealthyReplicationHandler {
+  private final PlacementPolicy placementPolicy;
+
+  protected AbstractOverReplicationHandler(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.

Review Comment:
   Missed to update doc?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/AbstractOverReplicationHandler.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * this class holds some common methods that will be shared among

Review Comment:
   Typos: this --> This
   Implementation -> implementation



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 commented on pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 commented on PR #3572:
URL: https://github.com/apache/ozone/pull/3572#issuecomment-1174508568

   thanks @umamaheswararao for the review! i have committed this patch to master branch


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] umamaheswararao commented on a diff in pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on code in PR #3572:
URL: https://github.com/apache/ozone/pull/3572#discussion_r912550235


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to delete the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for
+    // EC container and we always handle under-replicated first now. it
+    // means when reaching here, we have all the replica indexes and some
+    // of them are more than 1.
+    // TODO: support hybrid state if needed.
+    final ECContainerReplicaCount replicaCount =
+        new ECContainerReplicaCount(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+
+    List<Integer> overReplicatedIndexes =
+        replicaCount.overReplicatedIndexes(true);
+    //sanity check
+    if (overReplicatedIndexes.size() == 0) {
+      LOG.warn("The container {} with replicas {} is found over replicated " +
+              "by ContainerHealthCheck, but found not over replicated by " +
+              "ECContainerReplicaCount",
+          container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    final List<DatanodeDetails> deletionInFlight = new ArrayList<>();
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        deletionInFlight.add(op.getTarget());
+      }
+    }
+    Map<Integer, List<ContainerReplica>> index2replicas = new HashMap<>();
+    replicas.stream()
+        .filter(r -> overReplicatedIndexes.contains(r.getReplicaIndex()))
+        .filter(r -> r
+            .getState() == StorageContainerDatanodeProtocolProtos
+            .ContainerReplicaProto.State.CLOSED)
+        .filter(r -> {
+          DatanodeDetails dd = r.getDatanodeDetails();
+          try {

Review Comment:
   ReplicationManager has a static method to check the status. Let's use that?
   ReplicationManager
                   .getNodeStatus(r.getDatanodeDetails(), nodeManager).isHealthy())
                   
                   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 commented on a diff in pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 commented on code in PR #3572:
URL: https://github.com/apache/ozone/pull/3572#discussion_r912428114


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java:
##########
@@ -51,7 +50,7 @@
  * Handles the EC Under replication processing and forming the respective SCM
  * commands.
  */
-public class ECUnderReplicationHandler implements UnderReplicationHandler {
+public class ECUnderReplicationHandler implements UnhealthyReplicationHandler {

Review Comment:
   `ReplicationHandler` seems a little confusing, it might be considered as the handler to handle replication command.
   so i suggest we use `unhealthy` just for now and change it to an appropriate name after we find a proper one.  what about "nonOptimalReplicationHandler"



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 commented on pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 commented on PR #3572:
URL: https://github.com/apache/ozone/pull/3572#issuecomment-1173275872

   Thanks @umamaheswararao for the review , i have update this patch according to your comments, pleas take a look


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 commented on pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 commented on PR #3572:
URL: https://github.com/apache/ozone/pull/3572#issuecomment-1172988635

   @umamaheswararao  thanks for the review, will fix the comments later


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 merged pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 merged PR #3572:
URL: https://github.com/apache/ozone/pull/3572


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] umamaheswararao commented on a diff in pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
umamaheswararao commented on code in PR #3572:
URL: https://github.com/apache/ozone/pull/3572#discussion_r912403332


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for
+    // EC container , and we always handle under-replicated first now. it
+    // means when reaching here, we have all the replica indexes and some
+    // of them are more than 1.
+    // TODO: support hybrid state if needed.

Review Comment:
   remove this empty line



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for

Review Comment:
   typo: EC container , --> EC container,



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for
+    // EC container , and we always handle under-replicated first now. it
+    // means when reaching here, we have all the replica indexes and some
+    // of them are more than 1.
+    // TODO: support hybrid state if needed.
+
+    final ECContainerReplicaCount replicaCount =
+        new ECContainerReplicaCount(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+
+    List<Integer> overReplicatedIndexes =
+        replicaCount.overReplicatedIndexes(true);
+    //sanity check
+    if (overReplicatedIndexes.size() == 0) {
+      LOG.warn("The container {} with replicas {} is found over replicated " +
+              "by ContainerHealthCheck, but found not over replicated by " +
+              "ECContainerReplicaCount",
+          container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    final List<DatanodeDetails> deletionInFlight = new ArrayList<>();
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        deletionInFlight.add(op.getTarget());
+      }
+    }
+    Map<Integer, List<ContainerReplica>> index2replicas = new HashMap<>();
+    replicas.stream()
+        .filter(r -> overReplicatedIndexes.contains(r.getReplicaIndex()))
+        .filter(r -> r
+            .getState() == StorageContainerDatanodeProtocolProtos
+            .ContainerReplicaProto.State.CLOSED)
+        .filter(r -> {
+          DatanodeDetails dd = r.getDatanodeDetails();
+          try {
+            //the command target node should be in-service and healthy
+            return nodeManager.getNodeStatus(dd)
+                .equals(NodeStatus.inServiceHealthy());
+          } catch (NodeNotFoundException nnfe) {
+            //nothing to do, just skip;
+            LOG.warn("can not find node when getting NodeStatus, {}", dd);
+            return false;
+          }
+        }).filter(r -> !deletionInFlight.contains(r.getDatanodeDetails()))
+        .forEach(r -> {
+          int index = r.getReplicaIndex();
+          index2replicas.computeIfAbsent(index, k -> new LinkedList<>());
+          index2replicas.get(index).add(r);
+        });
+
+    if (index2replicas.size() > 0) {
+      final Map<DatanodeDetails, SCMCommand<?>> commands = new HashMap<>();
+      final int replicationFactor =
+          container.getReplicationConfig().getRequiredNodes();
+      index2replicas.values().forEach(l -> {
+        Iterator<ContainerReplica> it = l.iterator();
+        Set<ContainerReplica> tempReplicaSet = new HashSet<>();

Review Comment:
   we can pass replicas into ctor ot HashSet ?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for
+    // EC container , and we always handle under-replicated first now. it
+    // means when reaching here, we have all the replica indexes and some
+    // of them are more than 1.
+    // TODO: support hybrid state if needed.
+
+    final ECContainerReplicaCount replicaCount =
+        new ECContainerReplicaCount(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+
+    List<Integer> overReplicatedIndexes =
+        replicaCount.overReplicatedIndexes(true);
+    //sanity check
+    if (overReplicatedIndexes.size() == 0) {
+      LOG.warn("The container {} with replicas {} is found over replicated " +
+              "by ContainerHealthCheck, but found not over replicated by " +
+              "ECContainerReplicaCount",
+          container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    final List<DatanodeDetails> deletionInFlight = new ArrayList<>();
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        deletionInFlight.add(op.getTarget());
+      }
+    }
+    Map<Integer, List<ContainerReplica>> index2replicas = new HashMap<>();
+    replicas.stream()
+        .filter(r -> overReplicatedIndexes.contains(r.getReplicaIndex()))
+        .filter(r -> r
+            .getState() == StorageContainerDatanodeProtocolProtos
+            .ContainerReplicaProto.State.CLOSED)
+        .filter(r -> {
+          DatanodeDetails dd = r.getDatanodeDetails();
+          try {
+            //the command target node should be in-service and healthy
+            return nodeManager.getNodeStatus(dd)
+                .equals(NodeStatus.inServiceHealthy());
+          } catch (NodeNotFoundException nnfe) {
+            //nothing to do, just skip;
+            LOG.warn("can not find node when getting NodeStatus, {}", dd);
+            return false;
+          }
+        }).filter(r -> !deletionInFlight.contains(r.getDatanodeDetails()))
+        .forEach(r -> {
+          int index = r.getReplicaIndex();
+          index2replicas.computeIfAbsent(index, k -> new LinkedList<>());
+          index2replicas.get(index).add(r);
+        });
+
+    if (index2replicas.size() > 0) {
+      final Map<DatanodeDetails, SCMCommand<?>> commands = new HashMap<>();
+      final int replicationFactor =
+          container.getReplicationConfig().getRequiredNodes();
+      index2replicas.values().forEach(l -> {
+        Iterator<ContainerReplica> it = l.iterator();
+        Set<ContainerReplica> tempReplicaSet = new HashSet<>();
+        tempReplicaSet.addAll(replicas);
+        for (; it.hasNext() && l.size() > 1;) {

Review Comment:
   This could be while?



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.

Review Comment:
   Please fix this. This will generate only delete command, but not replicate/reconstruct commands.



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECOverReplicationHandler.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+
+/**
+ * Tests the ECOverReplicationHandling functionality.
+ */
+public class TestECOverReplicationHandler {
+  private ECReplicationConfig repConfig;
+  private ContainerInfo container;
+  private NodeManager nodeManager;
+  private OzoneConfiguration conf;
+  private PlacementPolicy policy;
+
+  @BeforeEach
+  public void setup() {
+    nodeManager = new MockNodeManager(true, 10) {
+      @Override
+      public NodeStatus getNodeStatus(DatanodeDetails dd)
+          throws NodeNotFoundException {
+        return NodeStatus.inServiceHealthy();
+      }
+    };
+    conf = SCMTestUtils.getConf();
+    repConfig = new ECReplicationConfig(3, 2);
+    container = ReplicationTestUtil
+        .createContainer(HddsProtos.LifeCycleState.CLOSED, repConfig);
+    policy = new SCMCommonPlacementPolicy(nodeManager, conf) {
+      @Override
+      public List<DatanodeDetails> chooseDatanodes(
+          List<DatanodeDetails> excludedNodes,
+          List<DatanodeDetails> favoredNodes, int nodesRequiredToChoose,
+          long metadataSizeRequired, long dataSizeRequired)
+          throws SCMException {
+        List<DatanodeDetails> dns = new ArrayList<>();
+        for (int i = 0; i < nodesRequiredToChoose; i++) {
+          dns.add(MockDatanodeDetails.randomDatanodeDetails());
+        }
+        return dns;
+      }
+
+      @Override
+      public DatanodeDetails chooseNode(List<DatanodeDetails> healthyNodes) {
+        return null;
+      }
+    };
+    NodeSchema[] schemas =
+        new NodeSchema[] {ROOT_SCHEMA, RACK_SCHEMA, LEAF_SCHEMA};
+    NodeSchemaManager.getInstance().init(schemas, true);
+  }
+
+  @Test

Review Comment:
   May be we can use parameterized tests for the following tests? I think we can do parameterized for TestUnderRplicationHandler also, but we can do that later as that was already committed. 



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECOverReplicationHandler.java:
##########
@@ -0,0 +1,178 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.util.Collections.emptyMap;
+
+/**
+ * Handles the EC Over replication processing and forming the respective SCM
+ * commands.
+ */
+public class ECOverReplicationHandler extends AbstractOverReplicationHandler {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(ECOverReplicationHandler.class);
+
+  private final ECContainerHealthCheck ecContainerHealthCheck =
+      new ECContainerHealthCheck();
+  private final NodeManager nodeManager;
+
+  public ECOverReplicationHandler(PlacementPolicy placementPolicy,
+                                  NodeManager nodeManager) {
+    super(placementPolicy);
+    this.nodeManager = nodeManager;
+  }
+
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.
+   *
+   * @param replicas - Set of available container replicas.
+   * @param pendingOps - Inflight replications and deletion ops.
+   * @param result - Health check result.
+   * @param remainingMaintenanceRedundancy - represents that how many nodes go
+   *                                      into maintenance.
+   * @return Returns the key value pair of destination dn where the command gets
+   * executed and the command itself.
+   */
+  @Override
+  public Map<DatanodeDetails, SCMCommand<?>> processAndCreateCommands(
+      Set<ContainerReplica> replicas, List<ContainerReplicaOp> pendingOps,
+      ContainerHealthResult result, int remainingMaintenanceRedundancy) {
+    ContainerInfo container = result.getContainerInfo();
+    ContainerHealthResult currentUnderRepRes = ecContainerHealthCheck
+        .checkHealth(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+    LOG.debug("Handling over-replicated EC container: {}", container);
+
+    //sanity check
+    if (currentUnderRepRes.getHealthState() !=
+        ContainerHealthResult.HealthState.OVER_REPLICATED) {
+      LOG.info("The container {} state changed and it's not in over"
+              + " replication any more. Current state is: {}",
+          container.getContainerID(), currentUnderRepRes);
+      return emptyMap();
+    }
+
+    ContainerHealthResult.OverReplicatedHealthResult containerHealthResult =
+        ((ContainerHealthResult.OverReplicatedHealthResult)
+            currentUnderRepRes);
+    if (containerHealthResult.isSufficientlyReplicatedAfterPending()) {
+      LOG.info("The container {} with replicas {} will be corrected " +
+              "by the pending delete", container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    // we don`t support hybrid state(both under and over replicated) for
+    // EC container , and we always handle under-replicated first now. it
+    // means when reaching here, we have all the replica indexes and some
+    // of them are more than 1.
+    // TODO: support hybrid state if needed.
+
+    final ECContainerReplicaCount replicaCount =
+        new ECContainerReplicaCount(container, replicas, pendingOps,
+            remainingMaintenanceRedundancy);
+
+    List<Integer> overReplicatedIndexes =
+        replicaCount.overReplicatedIndexes(true);
+    //sanity check
+    if (overReplicatedIndexes.size() == 0) {
+      LOG.warn("The container {} with replicas {} is found over replicated " +
+              "by ContainerHealthCheck, but found not over replicated by " +
+              "ECContainerReplicaCount",
+          container.getContainerID(), replicas);
+      return emptyMap();
+    }
+
+    final List<DatanodeDetails> deletionInFlight = new ArrayList<>();
+    for (ContainerReplicaOp op : pendingOps) {
+      if (op.getOpType() == ContainerReplicaOp.PendingOpType.DELETE) {
+        deletionInFlight.add(op.getTarget());
+      }
+    }
+    Map<Integer, List<ContainerReplica>> index2replicas = new HashMap<>();
+    replicas.stream()
+        .filter(r -> overReplicatedIndexes.contains(r.getReplicaIndex()))
+        .filter(r -> r
+            .getState() == StorageContainerDatanodeProtocolProtos
+            .ContainerReplicaProto.State.CLOSED)
+        .filter(r -> {
+          DatanodeDetails dd = r.getDatanodeDetails();
+          try {
+            //the command target node should be in-service and healthy
+            return nodeManager.getNodeStatus(dd)
+                .equals(NodeStatus.inServiceHealthy());

Review Comment:
   I think we discussed this. This is not covering healthy_readonly? 



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECOverReplicationHandler.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+
+/**
+ * Tests the ECOverReplicationHandling functionality.
+ */
+public class TestECOverReplicationHandler {
+  private ECReplicationConfig repConfig;
+  private ContainerInfo container;
+  private NodeManager nodeManager;
+  private OzoneConfiguration conf;
+  private PlacementPolicy policy;
+
+  @BeforeEach
+  public void setup() {
+    nodeManager = new MockNodeManager(true, 10) {
+      @Override
+      public NodeStatus getNodeStatus(DatanodeDetails dd)
+          throws NodeNotFoundException {
+        return NodeStatus.inServiceHealthy();
+      }
+    };
+    conf = SCMTestUtils.getConf();
+    repConfig = new ECReplicationConfig(3, 2);
+    container = ReplicationTestUtil
+        .createContainer(HddsProtos.LifeCycleState.CLOSED, repConfig);

Review Comment:
   May be something like getSimpleTestPlacementPolicy



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ECUnderReplicationHandler.java:
##########
@@ -51,7 +50,7 @@
  * Handles the EC Under replication processing and forming the respective SCM
  * commands.
  */
-public class ECUnderReplicationHandler implements UnderReplicationHandler {
+public class ECUnderReplicationHandler implements UnhealthyReplicationHandler {

Review Comment:
   Not sure we will call over replication as unhealthy. Don't have better name in my mind but how about simply ReplicationHandler ?
   
   
   Since it is common class now, javadoc should represent the generic message. Currently it assumed for underReplication handler.



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/TestECOverReplicationHandler.java:
##########
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.SCMCommonPlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.net.NodeSchema;
+import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
+import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
+
+/**
+ * Tests the ECOverReplicationHandling functionality.
+ */
+public class TestECOverReplicationHandler {
+  private ECReplicationConfig repConfig;
+  private ContainerInfo container;
+  private NodeManager nodeManager;
+  private OzoneConfiguration conf;
+  private PlacementPolicy policy;
+
+  @BeforeEach
+  public void setup() {
+    nodeManager = new MockNodeManager(true, 10) {
+      @Override
+      public NodeStatus getNodeStatus(DatanodeDetails dd)
+          throws NodeNotFoundException {
+        return NodeStatus.inServiceHealthy();
+      }
+    };
+    conf = SCMTestUtils.getConf();
+    repConfig = new ECReplicationConfig(3, 2);
+    container = ReplicationTestUtil
+        .createContainer(HddsProtos.LifeCycleState.CLOSED, repConfig);

Review Comment:
   Probably below code can be moved ReplicationTestUtil and reuse in both TestECUnderReplicationHandler and TestOverReplicationHandler ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] JacksonYao287 commented on a diff in pull request #3572: HDDS-6960. EC: Implement the Over-replication Handler

Posted by GitBox <gi...@apache.org>.
JacksonYao287 commented on code in PR #3572:
URL: https://github.com/apache/ozone/pull/3572#discussion_r912693462


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/AbstractOverReplicationHandler.java:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.container.replication;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
+import org.apache.hadoop.hdds.scm.PlacementPolicy;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * this class holds some common methods that will be shared among
+ * different kinds of Implementation of OverReplicationHandler.
+ * */
+public abstract class AbstractOverReplicationHandler
+    implements UnhealthyReplicationHandler {
+  private final PlacementPolicy placementPolicy;
+
+  protected AbstractOverReplicationHandler(PlacementPolicy placementPolicy) {
+    this.placementPolicy = placementPolicy;
+  }
+  /**
+   * Identify a new set of datanode(s) to replicate/reconstruct the container
+   * and form the SCM commands to send it to DN.

Review Comment:
   sorry for missing , fixed now!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org