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/10/25 08:12:31 UTC

[GitHub] [ozone] JacksonYao287 opened a new pull request, #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   ## What changes were proposed in this pull request?
   
   implement deleting container handler
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7384
   
   ## 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] JacksonYao287 commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   ok, i have updated tests , please 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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   @siddhantsangwan thanks for the review! i have updated this patch , please 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] DaveTeng0 commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */

Review Comment:
   ohh!! Yes! Thank you!!



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,

Review Comment:
   We can remove the containerManager and many of these other dependencies now we have moved the updateContainerState into the ReplicationManager I think.



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   To test this handler, we don't need to change the container state - we can just set it to the state we need when we created it and avoid the updateContainerState calls.



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(

Review Comment:
   Lets do it in this Jira please.



-- 
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] siddhantsangwan commented on pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   @JacksonYao287 Thanks for working on this. I'll review soon.


-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   Rather than using CM to allocated a container, just create a ContainerInfo with the correct state - there are some helper test classes in `ReplicationTestUtil` we have been using for this.



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(

Review Comment:
   If we remove the dependency on ContainerManager from the new handler, I think we can simplify these tests quite a bit, removing many of the dependencies.



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+  private final ContainerManager containerManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager,
+                                  ContainerManager containerManager) {
+    this.replicationManager = replicationManager;
+    this.containerManager = containerManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.
+   */
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    ContainerInfo containerInfo = request.getContainerInfo();
+    ContainerID cID = containerInfo.containerID();
+
+    if (containerInfo.getState() != HddsProtos.LifeCycleState.DELETING) {
+      return false;
+    }
+
+    if (request.getContainerReplicas().size() == 0) {
+      try {
+        containerManager.updateContainerState(

Review Comment:
   agree, and i notice `EmptyContainerhandler` also has this problem, so can we create a new jira for adding that proxy?



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+  private final ContainerManager containerManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager,
+                                  ContainerManager containerManager) {
+    this.replicationManager = replicationManager;
+    this.containerManager = containerManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.
+   */
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    ContainerInfo containerInfo = request.getContainerInfo();
+    ContainerID cID = containerInfo.containerID();
+
+    if (containerInfo.getState() != HddsProtos.LifeCycleState.DELETING) {

Review Comment:
   thanks for pointing out this , will add that check



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   here , we should verify that the container state is changed for DELETING to DELETED,  and we need this check to make sure the handler works as expected



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);
+  }
+
+
+  private void cleanupIfNoReplicaExist(
+      ReplicationConfig replicationConfig, int times) {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        replicationConfig, 1, DELETING);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Mockito.verify(replicationManager, Mockito.times(times))
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for some replica there is no pending delete, then resending delete
+   * command.
+   */
+  @Test
+  public void testResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    Set<ContainerReplica> tempContainerReplicas = new HashSet<>();
+    tempContainerReplicas.addAll(containerReplicas);
+    Iterator iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(2, tempContainerReplicas.size());
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 1);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    tempContainerReplicas.clear();
+    tempContainerReplicas.addAll(containerReplicas);
+    iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(3, tempContainerReplicas.size());
+
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));

Review Comment:
   thanks for the code!



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(

Review Comment:
   sure, can we do it in a new jira?



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,334 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,

Review Comment:
   removed



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestEmptyContainerHandler.java:
##########
@@ -63,9 +63,18 @@ public void setup()
         HddsProtos.ReplicationFactor.THREE);
     replicationManager = Mockito.mock(ReplicationManager.class);
     containerManager = Mockito.mock(ContainerManager.class);
+    
+    Mockito.doAnswer(invocation -> {

Review Comment:
   Do we need this part? If we remove the ContainerManager dependency, then we don't really need to have it in this test class anymore and we can just verify the ReplicationManager.updateContainerState is called and leave it there.



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+  private final ContainerManager containerManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager,
+                                  ContainerManager containerManager) {
+    this.replicationManager = replicationManager;
+    this.containerManager = containerManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.
+   */
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    ContainerInfo containerInfo = request.getContainerInfo();
+    ContainerID cID = containerInfo.containerID();
+
+    if (containerInfo.getState() != HddsProtos.LifeCycleState.DELETING) {
+      return false;
+    }
+
+    if (request.getContainerReplicas().size() == 0) {
+      try {
+        containerManager.updateContainerState(

Review Comment:
   We have been trying to avoid pushing "complex" dependencies into these handler classes. Instead, create a method on ReplicationManager, which already has a reference to ContainerManager, to "proxy" the request from the handler to the ContainerManager class.
   
   The advantage of this comes in testing, where we don't need to build up some many dependencies for a test, so it keeps the tests cleaner. We can simply mock the ReplicationManager and use Mockito to check the correct method has been called.



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);

Review Comment:
   yes , you are right. i will add a comment here to explain this



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   > @JacksonYao287 Changes look good. The test failure is in Replication Manager. Can you please check?
   
   this test successes locally on my laptop, i am not sure why it fails here. i retrigger CI to see whether it can be reproduced!


-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */

Review Comment:
   for now , a container is deleted means its state is changed to DELETED. it is not be removed from scm , so scm still has the reference of this container.
   
   #3360 this patch is trying to remove the reference from scm when its state is DELETED.
   
   I am not sure whether i have answered question. if not , please let me know!



-- 
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] sodonnel commented on pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   This change looks much better now. There is just a little more work to do to remove the ContainerManager dependency from the TestEmptyContainerHandler and then I think it is good.
   
   It would be good for @siddhantsangwan to have a look too, as he has been working in this area too recently.


-- 
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] siddhantsangwan merged pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


-- 
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] siddhantsangwan commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);

Review Comment:
   Is this 2 because it was already called once earlier for RATIS? It confused me at first. Maybe we can add a comment here explaining why it's 2.



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager) {
+    this.replicationManager = replicationManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state

Review Comment:
   NIT: We can change the comment to "If the number of replicas of the container is 0..."



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);
+  }
+
+
+  private void cleanupIfNoReplicaExist(
+      ReplicationConfig replicationConfig, int times) {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        replicationConfig, 1, DELETING);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Mockito.verify(replicationManager, Mockito.times(times))
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for some replica there is no pending delete, then resending delete
+   * command.
+   */
+  @Test
+  public void testResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    Set<ContainerReplica> tempContainerReplicas = new HashSet<>();
+    tempContainerReplicas.addAll(containerReplicas);
+    Iterator iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(2, tempContainerReplicas.size());
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 1);

Review Comment:
   This part is not very clear to me. Can you please explain what's being done here? 
   
   If we're trying add the DELETE op to only two replicas, something like this could be easier to understand:
   ```suggestion
       containerReplicas.stream().limit(2).forEach(replica -> pendingOps.add(
           ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
               replica.getDatanodeDetails(), replica.getReplicaIndex())));
   ```



##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager) {
+    this.replicationManager = replicationManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.

Review Comment:
   NIT: Since we are also handling DELETED containers here, the comment can be, "false if the specified container is not in DELETING or DELETED state..."



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);
+  }
+
+
+  private void cleanupIfNoReplicaExist(
+      ReplicationConfig replicationConfig, int times) {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        replicationConfig, 1, DELETING);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Mockito.verify(replicationManager, Mockito.times(times))
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for some replica there is no pending delete, then resending delete
+   * command.
+   */
+  @Test
+  public void testResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    Set<ContainerReplica> tempContainerReplicas = new HashSet<>();
+    tempContainerReplicas.addAll(containerReplicas);
+    Iterator iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(2, tempContainerReplicas.size());
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 1);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    tempContainerReplicas.clear();
+    tempContainerReplicas.addAll(containerReplicas);
+    iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(3, tempContainerReplicas.size());
+
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));

Review Comment:
   Same comment as above



##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist() {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE), 1);
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig, 2);
+  }
+
+
+  private void cleanupIfNoReplicaExist(
+      ReplicationConfig replicationConfig, int times) {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        replicationConfig, 1, DELETING);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Mockito.verify(replicationManager, Mockito.times(times))
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    containerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 0);
+
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for some replica there is no pending delete, then resending delete
+   * command.
+   */
+  @Test
+  public void testResendDeleteCommand() throws NotLeaderException {
+    //ratis container
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, DELETING);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 0, 0, 0);
+    List<ContainerReplicaOp> pendingOps = new ArrayList<>();
+    Set<ContainerReplica> tempContainerReplicas = new HashSet<>();
+    tempContainerReplicas.addAll(containerReplicas);
+    Iterator iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(2, tempContainerReplicas.size());
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 1);
+
+    //EC container
+    containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, DELETING);
+    containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSED, 1, 2, 3, 4, 5);
+    pendingOps.clear();
+    tempContainerReplicas.clear();
+    tempContainerReplicas.addAll(containerReplicas);
+    iter = tempContainerReplicas.iterator();
+    iter.next();
+    iter.remove();
+    iter.next();
+    iter.remove();
+    Assert.assertEquals(3, tempContainerReplicas.size());
+
+    tempContainerReplicas.forEach(r -> pendingOps.add(
+        ContainerReplicaOp.create(ContainerReplicaOp.PendingOpType.DELETE,
+            r.getDatanodeDetails(), r.getReplicaIndex())));
+    //since one delete command is end when testing ratis container, so
+    //here should be 1+2 = 3 times
+    resendDeleteCommand(containerInfo, containerReplicas, pendingOps, 3);
+
+  }
+
+  private void resendDeleteCommand(ContainerInfo containerInfo,

Review Comment:
   Perhaps we can rename this method to `verifyDeleteCommandCount`



-- 
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] siddhantsangwan commented on pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   @JacksonYao287 Changes look good. The test failure is in Replication Manager. Can you please check?


-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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

   @siddhantsangwan PTAL!
   


-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestEmptyContainerHandler.java:
##########
@@ -63,9 +63,18 @@ public void setup()
         HddsProtos.ReplicationFactor.THREE);
     replicationManager = Mockito.mock(ReplicationManager.class);
     containerManager = Mockito.mock(ContainerManager.class);
+    
+    Mockito.doAnswer(invocation -> {

Review Comment:
   thanks, removed containerManager from the 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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+  private final ContainerManager containerManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager,
+                                  ContainerManager containerManager) {
+    this.replicationManager = replicationManager;
+    this.containerManager = containerManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.
+   */
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    ContainerInfo containerInfo = request.getContainerInfo();
+    ContainerID cID = containerInfo.containerID();
+
+    if (containerInfo.getState() != HddsProtos.LifeCycleState.DELETING) {

Review Comment:
   Should we add here a check like:
   
   ```
   if (containerInfo.getState() == HddsProtos.LifeCycleState.DELETED) {
         return true;
   }
   ```
   
   So that it stops any further handlers running? I know we would like to remove DELETED containers from the system, but so far that is not agreed.



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   No we don't. These are unit tests and testing the transition of the container is another units job. We just need to test the correct method is called with the correct parameters on ReplicationManager and then from a unit testing perspective, we know the handler is doing the correct thing.



-- 
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] DaveTeng0 commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */

Review Comment:
   hmm~ just for myself learning~~ is it possible something goes wrong in Deleting state, then the container is not deleted successfully?



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerManagerImpl;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaPendingOps;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManager;
+import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
+import org.apache.hadoop.hdds.scm.ha.SequenceIdGenerator;
+import org.apache.hadoop.hdds.scm.metadata.SCMDBDefinition;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.pipeline.MockPipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.hdds.utils.db.DBStoreBuilder;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.hadoop.ozone.container.common.SCMTestUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETED;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private File testDir;
+  private ContainerManager containerManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+  private ContainerReplicaPendingOps pendingOpsMock;
+  private SequenceIdGenerator sequenceIdGen;
+  private SCMHAManager scmhaManager;
+  private NodeManager nodeManager;
+  private PipelineManager pipelineManager;
+  private DBStore dbStore;
+
+  @BeforeEach
+  public void setup() throws IOException {
+    final OzoneConfiguration conf = SCMTestUtils.getConf();
+    testDir = GenericTestUtils.getTestDir(
+        TestDeletingContainerHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
+    dbStore = DBStoreBuilder.createDBStore(
+        conf, new SCMDBDefinition());
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    scmhaManager = SCMHAManagerStub.getInstance(true);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+    pendingOpsMock = Mockito.mock(ContainerReplicaPendingOps.class);
+    nodeManager = new MockNodeManager(true, 10);
+    sequenceIdGen = new SequenceIdGenerator(
+        conf, scmhaManager, SCMDBDefinition.SEQUENCE_ID.getTable(dbStore));
+    pipelineManager =
+        new MockPipelineManager(dbStore, scmhaManager, nodeManager);
+    containerManager = new ContainerManagerImpl(conf,
+        scmhaManager, sequenceIdGen, pipelineManager,
+        SCMDBDefinition.CONTAINERS.getTable(dbStore), pendingOpsMock);
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager, containerManager);
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (containerManager != null) {
+      containerManager.close();
+    }
+
+    if (dbStore != null) {
+      dbStore.close();
+    }
+
+    FileUtil.fullyDelete(testDir);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */
+  @Test
+  public void testCleanupIfNoReplicaExist()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    cleanupIfNoReplicaExist(RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE));
+
+    //ec container
+    cleanupIfNoReplicaExist(ecReplicationConfig);
+  }
+
+
+  private void cleanupIfNoReplicaExist(ReplicationConfig replicationConfig)
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    ContainerInfo containerInfo = containerManager.allocateContainer(
+        replicationConfig, "admin");
+    ContainerID cID = containerInfo.containerID();
+
+    //change the state of the container to Deleting
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.FINALIZE);
+    containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
+    containerManager.updateContainerState(cID,
+        HddsProtos.LifeCycleEvent.DELETE);
+
+    Set<ContainerReplica> containerReplicas = new HashSet<>();
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertTrue(deletingContainerHandler.handle(request));
+    Assert.assertTrue(containerInfo.getState() == DELETED);
+  }
+
+  /**
+   * If a container is in Deleting state , some replicas exist and
+   * for each replica there is a pending delete, then do nothing.
+   */
+  @Test
+  public void testNoNeedResendDeleteCommand()
+      throws IOException, TimeoutException, InvalidStateTransitionException {
+    //ratis container
+    ContainerInfo containerInfo = containerManager.allocateContainer(

Review Comment:
   i used `ReplicationTestUtil` origianlly, but there will be a problem casued by `lifeCycleStateMap` when firing an event to change the state of a container, so i use containerManager#allocateContainer directly!
   ```
       //change the state of the container to Deleting
       containerManager.updateContainerState(cID,
           HddsProtos.LifeCycleEvent.FINALIZE);
       containerManager.updateContainerState(cID, HddsProtos.LifeCycleEvent.CLOSE);
       containerManager.updateContainerState(cID,
           HddsProtos.LifeCycleEvent.DELETE);
   ```



-- 
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] sodonnel commented on a diff in pull request #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+  private final ContainerManager containerManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager,
+                                  ContainerManager containerManager) {
+    this.replicationManager = replicationManager;
+    this.containerManager = containerManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.
+   */
+  @Override
+  public boolean handle(ContainerCheckRequest request) {
+    ContainerInfo containerInfo = request.getContainerInfo();
+    ContainerID cID = containerInfo.containerID();
+
+    if (containerInfo.getState() != HddsProtos.LifeCycleState.DELETING) {
+      return false;
+    }
+
+    if (request.getContainerReplicas().size() == 0) {
+      try {
+        containerManager.updateContainerState(

Review Comment:
   This is just a method on the Replication Manger class. I think we should do it on this Jira rather than splitting such a small task into another one.



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager) {
+    this.replicationManager = replicationManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.

Review Comment:
   if it is DELETED, we should return true so that later handler will not be invoked.
   
   please see https://github.com/apache/ozone/pull/3881#discussion_r1004481723



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/health/DeletingContainerHandler.java:
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Class used in Replication Manager to handle the
+ * replicas of containers in DELETING State.
+ */
+public class DeletingContainerHandler extends AbstractCheck {
+  private final ReplicationManager replicationManager;
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(DeletingContainerHandler.class);
+
+  public DeletingContainerHandler(ReplicationManager replicationManager) {
+    this.replicationManager = replicationManager;
+  }
+
+  /**
+   * If the replica size of the container is 0, change the state
+   * of the container to Deleted, otherwise resend delete command if needed.
+   * @param request ContainerCheckRequest object representing the container
+   * @return false if the specified container is not in DELETING state,
+   * otherwise true.

Review Comment:
   if it is DELETED, we should return true so that later handler will not be invoked



-- 
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 #3881: HDDS-7384. EC: ReplicationManager - implement deleting container handler

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


##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/replication/health/TestDeletingContainerHandler.java:
##########
@@ -0,0 +1,238 @@
+/*
+ * 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.health;
+
+import org.apache.hadoop.hdds.client.ECReplicationConfig;
+import org.apache.hadoop.hdds.client.RatisReplicationConfig;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerCheckRequest;
+import org.apache.hadoop.hdds.scm.container.replication.ContainerReplicaOp;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
+import org.apache.hadoop.hdds.scm.container.replication.ReplicationTestUtil;
+import org.apache.ratis.protocol.exceptions.NotLeaderException;
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState.DELETING;
+
+/**
+ * Tests for {@link DeletingContainerHandler}.
+ */
+public class TestDeletingContainerHandler {
+  private ReplicationManager replicationManager;
+  private DeletingContainerHandler deletingContainerHandler;
+  private ECReplicationConfig ecReplicationConfig;
+  private RatisReplicationConfig ratisReplicationConfig;
+
+
+  @BeforeEach
+  public void setup() throws IOException {
+
+    ecReplicationConfig = new ECReplicationConfig(3, 2);
+    ratisReplicationConfig = RatisReplicationConfig.getInstance(
+        HddsProtos.ReplicationFactor.THREE);
+    replicationManager = Mockito.mock(ReplicationManager.class);
+
+    Mockito.doNothing().when(replicationManager)
+        .updateContainerState(Mockito.any(ContainerID.class),
+            Mockito.any(HddsProtos.LifeCycleEvent.class));
+
+    deletingContainerHandler =
+        new DeletingContainerHandler(replicationManager);
+  }
+
+  /**
+   * If a container is not in Deleting state, it should not be handled by
+   * DeletingContainerHandler. It should return false so the request can be
+   * passed to the next handler in the chain.
+   */
+  @Test
+  public void testNonDeletingContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ecReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 1, 2, 3, 4, 5);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  @Test
+  public void testNonDeletingRatisContainerReturnsFalse() {
+    ContainerInfo containerInfo = ReplicationTestUtil.createContainerInfo(
+        ratisReplicationConfig, 1, CLOSED);
+    Set<ContainerReplica> containerReplicas = ReplicationTestUtil
+        .createReplicas(containerInfo.containerID(),
+            ContainerReplicaProto.State.CLOSING, 0, 0, 0);
+
+    ContainerCheckRequest request = new ContainerCheckRequest.Builder()
+        .setPendingOps(Collections.EMPTY_LIST)
+        .setReport(new ReplicationManagerReport())
+        .setContainerInfo(containerInfo)
+        .setContainerReplicas(containerReplicas)
+        .build();
+
+    Assert.assertFalse(deletingContainerHandler.handle(request));
+  }
+
+  /**
+   * If a container is in Deleting state and no replica exists,
+   * change the state of the container to DELETED.
+   */

Review Comment:
   for now , a container is deleted means its state is changed to DELETED. it is not be removed from scm , so scm still has the reference of this container.
   
   #3360 this patch is trying to remove the reference from scm when its state is DELETED.
   
   I am not sure whether i have answered your question. if not , please let me know!



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