You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by av...@apache.org on 2021/06/07 18:16:52 UTC

[ozone] branch master updated: HDDS-5235. Add precheck to avoid ContainerHealthTask setting Deleted containers to Missing state (#2250)

This is an automated email from the ASF dual-hosted git repository.

avijayan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 3d3a87f  HDDS-5235. Add precheck to avoid ContainerHealthTask setting Deleted containers to Missing state (#2250)
3d3a87f is described below

commit 3d3a87f61c037eda3e64dafa12214e5648f3894b
Author: Symious <yi...@foxmail.com>
AuthorDate: Tue Jun 8 02:16:37 2021 +0800

    HDDS-5235. Add precheck to avoid ContainerHealthTask setting Deleted containers to Missing state (#2250)
---
 .../ozone/recon/fsck/ContainerHealthStatus.java    |  6 ++
 .../ozone/recon/fsck/ContainerHealthTask.java      | 51 ++++++++++++-
 .../scm/ReconStorageContainerManagerFacade.java    |  5 ++
 .../ozone/recon/fsck/TestContainerHealthTask.java  | 88 ++++++++++++++++++++++
 4 files changed, 148 insertions(+), 2 deletions(-)

diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthStatus.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthStatus.java
index 472efb4..be0b2ce 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthStatus.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthStatus.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.ozone.recon.fsck;
 
 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.ContainerPlacementStatus;
 import org.apache.hadoop.hdds.scm.PlacementPolicy;
@@ -68,6 +69,11 @@ public class ContainerHealthStatus {
     return replicaDelta == 0 && !isMisReplicated();
   }
 
+  public boolean isDeleted() {
+    return container.getState() == HddsProtos.LifeCycleState.DELETED ||
+        container.getState() == HddsProtos.LifeCycleState.DELETING;
+  }
+
   public boolean isOverReplicated() {
     return replicaDelta < 0;
   }
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
index 04afed2..5f4df00 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java
@@ -18,18 +18,24 @@
 
 package org.apache.hadoop.ozone.recon.fsck;
 
+import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.PlacementPolicy;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerManagerV2;
 import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
 import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
+import org.apache.hadoop.ozone.common.statemachine.InvalidStateTransitionException;
 import org.apache.hadoop.ozone.recon.persistence.ContainerHealthSchemaManager;
 import org.apache.hadoop.ozone.recon.scm.ReconScmTask;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
 import org.apache.hadoop.ozone.recon.tasks.ReconTaskConfig;
 import org.apache.hadoop.util.Time;
 import org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates;
@@ -49,6 +55,7 @@ public class ContainerHealthTask extends ReconScmTask {
   private static final Logger LOG =
       LoggerFactory.getLogger(ContainerHealthTask.class);
 
+  private StorageContainerServiceProvider scmClient;
   private ContainerManagerV2 containerManager;
   private ContainerHealthSchemaManager containerHealthSchemaManager;
   private PlacementPolicy placementPolicy;
@@ -57,11 +64,13 @@ public class ContainerHealthTask extends ReconScmTask {
 
   public ContainerHealthTask(
       ContainerManagerV2 containerManager,
+      StorageContainerServiceProvider scmClient,
       ReconTaskStatusDao reconTaskStatusDao,
       ContainerHealthSchemaManager containerHealthSchemaManager,
       PlacementPolicy placementPolicy,
       ReconTaskConfig reconTaskConfig) {
     super(reconTaskStatusDao);
+    this.scmClient = scmClient;
     this.containerHealthSchemaManager = containerHealthSchemaManager;
     this.placementPolicy = placementPolicy;
     this.containerManager = containerManager;
@@ -147,6 +156,11 @@ public class ContainerHealthTask extends ReconScmTask {
           }
           if (ContainerHealthRecords
               .retainOrUpdateRecord(currentContainer, rec)) {
+            // Check if the missing container is deleted in SCM
+            if (currentContainer.isMissing() &&
+                containerDeletedInSCM(currentContainer.getContainer())) {
+              rec.delete();
+            }
             existingRecords.add(rec.getContainerState());
             if (rec.changed()) {
               rec.update();
@@ -174,7 +188,11 @@ public class ContainerHealthTask extends ReconScmTask {
           containerManager.getContainerReplicas(container.containerID());
       ContainerHealthStatus h = new ContainerHealthStatus(
           container, containerReplicas, placementPolicy);
-      if (h.isHealthy()) {
+      if (h.isHealthy() || h.isDeleted()) {
+        return;
+      }
+      // For containers deleted in SCM, we sync the container state here.
+      if (h.isMissing() && containerDeletedInSCM(container)) {
         return;
       }
       containerHealthSchemaManager.insertUnhealthyContainerRecords(
@@ -185,6 +203,35 @@ public class ContainerHealthTask extends ReconScmTask {
     }
   }
 
+  private boolean containerDeletedInSCM(ContainerInfo containerInfo) {
+    try {
+      ContainerWithPipeline containerWithPipeline =
+          scmClient.getContainerWithPipeline(containerInfo.getContainerID());
+      if (containerWithPipeline.getContainerInfo().getState() ==
+          HddsProtos.LifeCycleState.DELETED) {
+        if (containerInfo.getState() == HddsProtos.LifeCycleState.CLOSED) {
+          containerManager.updateContainerState(containerInfo.containerID(),
+              HddsProtos.LifeCycleEvent.DELETE);
+        }
+        if (containerInfo.getState() == HddsProtos.LifeCycleState.DELETING &&
+            containerManager.getContainerReplicas(containerInfo.containerID())
+                .size() == 0
+        ) {
+          containerManager.updateContainerState(containerInfo.containerID(),
+              HddsProtos.LifeCycleEvent.CLEANUP);
+        }
+        return true;
+      }
+    } catch (InvalidStateTransitionException e) {
+      LOG.error("Failed to transition Container state while processing " +
+          "container in Container Health task", e);
+    } catch (IOException e) {
+      LOG.error("Got exception while processing container in" +
+          " Container Health task", e);
+    }
+    return false;
+  }
+
   /**
    * Helper methods to generate and update the required database records for
    * unhealthy containers.
@@ -245,7 +292,7 @@ public class ContainerHealthTask extends ReconScmTask {
         ContainerHealthStatus container, Set<String> recordForStateExists,
         long time) {
       List<UnhealthyContainers> records = new ArrayList<>();
-      if (container.isHealthy()) {
+      if (container.isHealthy() || container.isDeleted()) {
         return records;
       }
 
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
index c7f4e22..ebef4d8 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java
@@ -195,6 +195,7 @@ public class ReconStorageContainerManagerFacade
         reconTaskConfig));
     reconScmTasks.add(new ContainerHealthTask(
         containerManager,
+        scmServiceProvider,
         reconTaskStatusDao, containerHealthSchemaManager,
         containerPlacementPolicy,
         reconTaskConfig));
@@ -334,4 +335,8 @@ public class ReconStorageContainerManagerFacade
   public EventQueue getEventQueue() {
     return eventQueue;
   }
+
+  public StorageContainerServiceProvider getScmServiceProvider() {
+    return scmServiceProvider;
+  }
 }
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java
index 18f9fa4..18125ea 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java
@@ -42,9 +42,11 @@ import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerManagerV2;
 import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
 import org.apache.hadoop.ozone.recon.persistence.ContainerHealthSchemaManager;
 import org.apache.hadoop.ozone.recon.scm.ReconStorageContainerManagerFacade;
+import org.apache.hadoop.ozone.recon.spi.StorageContainerServiceProvider;
 import org.apache.hadoop.ozone.recon.tasks.ReconTaskConfig;
 import org.apache.ozone.test.LambdaTestUtils;
 import org.hadoop.ozone.recon.schema.ContainerSchemaDefinition;
@@ -73,6 +75,8 @@ public class TestContainerHealthTask extends AbstractReconSqlDBTest {
         mock(ReconStorageContainerManagerFacade.class);
     MockPlacementPolicy placementMock = new MockPlacementPolicy();
     ContainerManagerV2 containerManagerMock = mock(ContainerManagerV2.class);
+    StorageContainerServiceProvider scmClientMock =
+        mock(StorageContainerServiceProvider.class);
     ContainerReplica unhealthyReplicaMock = mock(ContainerReplica.class);
     when(unhealthyReplicaMock.getState()).thenReturn(State.UNHEALTHY);
     ContainerReplica healthyReplicaMock = mock(ContainerReplica.class);
@@ -81,10 +85,13 @@ public class TestContainerHealthTask extends AbstractReconSqlDBTest {
     // Create 6 containers. The first 5 will have various unhealthy states
     // defined below. The container with ID=6 will be healthy.
     List<ContainerInfo> mockContainers = getMockContainers(6);
+    when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock);
     when(scmMock.getContainerManager()).thenReturn(containerManagerMock);
     when(containerManagerMock.getContainers()).thenReturn(mockContainers);
     for (ContainerInfo c : mockContainers) {
       when(containerManagerMock.getContainer(c.containerID())).thenReturn(c);
+      when(scmClientMock.getContainerWithPipeline(c.getContainerID()))
+          .thenReturn(new ContainerWithPipeline(c, null));
     }
     // Under replicated
     when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(1L)))
@@ -125,6 +132,7 @@ public class TestContainerHealthTask extends AbstractReconSqlDBTest {
     reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(2));
     ContainerHealthTask containerHealthTask =
         new ContainerHealthTask(scmMock.getContainerManager(),
+            scmMock.getScmServiceProvider(),
             reconTaskStatusDao, containerHealthSchemaManager,
             placementMock, reconTaskConfig);
     containerHealthTask.start();
@@ -206,6 +214,76 @@ public class TestContainerHealthTask extends AbstractReconSqlDBTest {
         unHealthyContainersTableHandle.fetchByContainerId(5L).size());
   }
 
+  @Test
+  public void testDeletedContainer() throws Exception {
+    UnhealthyContainersDao unHealthyContainersTableHandle =
+        getDao(UnhealthyContainersDao.class);
+
+    ContainerHealthSchemaManager containerHealthSchemaManager =
+        new ContainerHealthSchemaManager(
+            getSchemaDefinition(ContainerSchemaDefinition.class),
+            unHealthyContainersTableHandle);
+    ReconStorageContainerManagerFacade scmMock =
+        mock(ReconStorageContainerManagerFacade.class);
+    MockPlacementPolicy placementMock = new MockPlacementPolicy();
+    ContainerManagerV2 containerManagerMock = mock(ContainerManagerV2.class);
+    StorageContainerServiceProvider scmClientMock =
+        mock(StorageContainerServiceProvider.class);
+
+    // Create 2 containers. The first is OPEN will no replicas, the second is
+    // CLOSED with no replicas.
+    List<ContainerInfo> mockContainers = getMockContainers(2);
+    when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock);
+    when(scmMock.getContainerManager()).thenReturn(containerManagerMock);
+    when(containerManagerMock.getContainers()).thenReturn(mockContainers);
+    for (ContainerInfo c : mockContainers) {
+      when(containerManagerMock.getContainer(c.containerID())).thenReturn(c);
+      when(scmClientMock.getContainerWithPipeline(c.getContainerID()))
+          .thenReturn(new ContainerWithPipeline(c, null));
+    }
+    // Container State OPEN with no replicas
+    when(containerManagerMock.getContainer(ContainerID.valueOf(1L)).getState())
+        .thenReturn(HddsProtos.LifeCycleState.OPEN);
+    when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(1L)))
+        .thenReturn(Collections.emptySet());
+    when(scmClientMock.getContainerWithPipeline(1))
+        .thenReturn(new ContainerWithPipeline(mockContainers.get(0), null));
+
+    // Container State CLOSED with no replicas
+    when(containerManagerMock.getContainer(ContainerID.valueOf(2L)).getState())
+        .thenReturn(HddsProtos.LifeCycleState.CLOSED);
+    when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(2L)))
+        .thenReturn(Collections.emptySet());
+    ContainerInfo mockDeletedContainer = getMockDeletedContainer(2);
+    when(scmClientMock.getContainerWithPipeline(2))
+        .thenReturn(new ContainerWithPipeline(mockDeletedContainer, null));
+
+    List<UnhealthyContainers> all = unHealthyContainersTableHandle.findAll();
+    Assert.assertTrue(all.isEmpty());
+
+    long currentTime = System.currentTimeMillis();
+    ReconTaskStatusDao reconTaskStatusDao = getDao(ReconTaskStatusDao.class);
+    ReconTaskConfig reconTaskConfig = new ReconTaskConfig();
+    reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(2));
+    ContainerHealthTask containerHealthTask =
+        new ContainerHealthTask(scmMock.getContainerManager(),
+            scmMock.getScmServiceProvider(),
+            reconTaskStatusDao, containerHealthSchemaManager,
+            placementMock, reconTaskConfig);
+    containerHealthTask.start();
+    LambdaTestUtils.await(6000, 1000, () ->
+        (unHealthyContainersTableHandle.count() == 1));
+    UnhealthyContainers rec =
+        unHealthyContainersTableHandle.fetchByContainerId(1L).get(0);
+    assertEquals("MISSING", rec.getContainerState());
+    assertEquals(3, rec.getReplicaDelta().intValue());
+
+    ReconTaskStatus taskStatus =
+        reconTaskStatusDao.findById(containerHealthTask.getTaskName());
+    Assert.assertTrue(taskStatus.getLastUpdatedTimestamp() >
+        currentTime);
+  }
+
   private Set<ContainerReplica> getMockReplicas(
       long containerId, State...states) {
     Set<ContainerReplica> replicas = new HashSet<>();
@@ -233,6 +311,16 @@ public class TestContainerHealthTask extends AbstractReconSqlDBTest {
     return containers;
   }
 
+  private ContainerInfo getMockDeletedContainer(int containerID) {
+    ContainerInfo c = mock(ContainerInfo.class);
+    when(c.getContainerID()).thenReturn((long)containerID);
+    when(c.getReplicationFactor())
+        .thenReturn(HddsProtos.ReplicationFactor.THREE);
+    when(c.containerID()).thenReturn(ContainerID.valueOf(containerID));
+    when(c.getState()).thenReturn(HddsProtos.LifeCycleState.DELETED);
+    return c;
+  }
+
   /**
    * This is a simple implementation of PlacementPolicy, so that when
    * validateContainerPlacement() is called, by default it will return a value

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