HDDS-642. Add chill mode exit condition for pipeline availability. Contributed 
by Yiqun Lin.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b71cc7f3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b71cc7f3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b71cc7f3

Branch: refs/heads/HDFS-12943
Commit: b71cc7f33edbbf6a98d1efb330f1c748b5dd6e75
Parents: efc4d91
Author: Ajay Kumar <a...@apache.org>
Authored: Wed Nov 28 17:45:46 2018 -0800
Committer: Ajay Kumar <a...@apache.org>
Committed: Wed Nov 28 17:47:57 2018 -0800

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/HddsConfigKeys.java  |   5 +
 .../common/src/main/resources/ozone-default.xml |   9 ++
 .../scm/chillmode/PipelineChillModeRule.java    | 108 +++++++++++++++++++
 .../hdds/scm/chillmode/SCMChillModeManager.java |  19 +++-
 .../scm/server/StorageContainerManager.java     |   5 +-
 .../scm/chillmode/TestSCMChillModeManager.java  |  81 ++++++++++++--
 6 files changed, 213 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 2d28a5b..f16503e 100644
--- 
a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ 
b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -87,6 +87,11 @@ public final class HddsConfigKeys {
       "hdds.scm.chillmode.min.datanode";
   public static final int HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT = 1;
 
+  public static final String HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK =
+      "hdds.scm.chillmode.pipeline-availability.check";
+  public static final boolean
+      HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT = false;
+
   // % of containers which should have at least one reported replica
   // before SCM comes out of chill mode.
   public static final String HDDS_SCM_CHILLMODE_THRESHOLD_PCT =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/common/src/main/resources/ozone-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml 
b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 9f3d7e1..aa22b2b 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1232,6 +1232,15 @@
   </property>
 
   <property>
+    <name>hdds.scm.chillmode.pipeline-availability.check</name>
+    <value>false</value>
+    <tag>HDDS,SCM,OPERATION</tag>
+    <description>
+      Boolean value to enable pipeline availability check during SCM chill 
mode.
+    </description>
+  </property>
+
+  <property>
     <name>hdds.container.action.max.limit</name>
     <value>20</value>
     <tag>DATANODE</tag>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/PipelineChillModeRule.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/PipelineChillModeRule.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/PipelineChillModeRule.java
new file mode 100644
index 0000000..f9a6e59
--- /dev/null
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/PipelineChillModeRule.java
@@ -0,0 +1,108 @@
+/**
+ * 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.chillmode;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
+import 
org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
+import org.apache.hadoop.hdds.server.events.EventHandler;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Class defining Chill mode exit criteria for Pipelines.
+ */
+public class PipelineChillModeRule
+    implements ChillModeExitRule<PipelineReportFromDatanode>,
+    EventHandler<PipelineReportFromDatanode> {
+  /** Pipeline availability.*/
+  private AtomicBoolean isPipelineAvailable = new AtomicBoolean(false);
+
+  private final PipelineManager pipelineManager;
+  private final SCMChillModeManager chillModeManager;
+
+  PipelineChillModeRule(PipelineManager pipelineManager,
+      SCMChillModeManager manager) {
+    this.pipelineManager = pipelineManager;
+    this.chillModeManager = manager;
+  }
+
+  @Override
+  public boolean validate() {
+    return isPipelineAvailable.get();
+  }
+
+  @Override
+  public void process(PipelineReportFromDatanode report) {
+    // No need to deal with
+  }
+
+  @Override
+  public void cleanup() {
+    // No need to deal with
+  }
+
+  @Override
+  public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
+      EventPublisher publisher) {
+    // If we are already in pipeline available state,
+    // skipping following check.
+    if (validate()) {
+      chillModeManager.validateChillModeExitRules(publisher);
+      return;
+    }
+
+    Pipeline pipeline;
+    Preconditions.checkNotNull(pipelineReportFromDatanode);
+    PipelineReportsProto pipelineReport = pipelineReportFromDatanode
+        .getReport();
+
+    for (PipelineReport report : pipelineReport.getPipelineReportList()) {
+      PipelineID pipelineID = PipelineID
+          .getFromProtobuf(report.getPipelineID());
+      try {
+        pipeline = pipelineManager.getPipeline(pipelineID);
+      } catch (PipelineNotFoundException e) {
+        continue;
+      }
+
+      if (pipeline.getPipelineState() == Pipeline.PipelineState.OPEN) {
+        // ensure there is an OPEN state pipeline and then allowed
+        // to exit chill mode
+        isPipelineAvailable.set(true);
+
+        if (chillModeManager.getInChillMode()) {
+          SCMChillModeManager.getLogger()
+              .info("SCM in chill mode. 1 Pipeline reported, 1 required.");
+        }
+        break;
+      }
+    }
+
+    if (validate()) {
+      chillModeManager.validateChillModeExitRules(publisher);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
index fa0eaa9..09b998d 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
     .NodeRegistrationContainerReport;
 import org.apache.hadoop.hdds.server.events.EventHandler;
@@ -53,14 +54,18 @@ public class SCMChillModeManager implements
       LoggerFactory.getLogger(SCMChillModeManager.class);
   private final boolean isChillModeEnabled;
   private AtomicBoolean inChillMode = new AtomicBoolean(true);
+
   private Map<String, ChillModeExitRule> exitRules = new HashMap(1);
   private Configuration config;
   private static final String CONT_EXIT_RULE = "ContainerChillModeRule";
   private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
+  private static final String PIPELINE_EXIT_RULE = "PipelineChillModeRule";
+
   private final EventQueue eventPublisher;
 
   public SCMChillModeManager(Configuration conf,
-      List<ContainerInfo> allContainers, EventQueue eventQueue) {
+      List<ContainerInfo> allContainers, PipelineManager pipelineManager,
+      EventQueue eventQueue) {
     this.config = conf;
     this.eventPublisher = eventQueue;
     this.isChillModeEnabled = conf.getBoolean(
@@ -70,6 +75,16 @@ public class SCMChillModeManager implements
       exitRules.put(CONT_EXIT_RULE,
           new ContainerChillModeRule(config, allContainers, this));
       exitRules.put(DN_EXIT_RULE, new DataNodeChillModeRule(config, this));
+
+      if (conf.getBoolean(
+          HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
+          
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
+          && pipelineManager != null) {
+        PipelineChillModeRule rule = new PipelineChillModeRule(pipelineManager,
+            this);
+        exitRules.put(PIPELINE_EXIT_RULE, rule);
+        eventPublisher.addHandler(SCMEvents.PIPELINE_REPORT, rule);
+      }
       emitChillModeStatus();
     } else {
       exitChillMode(eventQueue);
@@ -84,7 +99,7 @@ public class SCMChillModeManager implements
     eventPublisher.fireEvent(SCMEvents.CHILL_MODE_STATUS, getInChillMode());
   }
 
-  private void validateChillModeExitRules(EventPublisher eventQueue) {
+  public void validateChillModeExitRules(EventPublisher eventQueue) {
     for (ChillModeExitRule exitRule : exitRules.values()) {
       if (!exitRule.validate()) {
         return;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index fe00b54..2d27984 100644
--- 
a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ 
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -217,7 +217,7 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
     NodeReportHandler nodeReportHandler =
         new NodeReportHandler(scmNodeManager);
     PipelineReportHandler pipelineReportHandler =
-            new PipelineReportHandler(pipelineManager, conf);
+        new PipelineReportHandler(pipelineManager, conf);
     CommandStatusReportHandler cmdStatusReportHandler =
         new CommandStatusReportHandler();
 
@@ -302,8 +302,7 @@ public final class StorageContainerManager extends 
ServiceRuntimeInfoImpl
     eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
         (BlockManagerImpl) scmBlockManager);
     scmChillModeManager = new SCMChillModeManager(conf,
-        containerManager.getContainers(),
-        eventQueue);
+        containerManager.getContainers(), pipelineManager, eventQueue);
 
     eventQueue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
         scmChillModeManager);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b71cc7f3/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
----------------------------------------------------------------------
diff --git 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
index 0487fb7..ed17796 100644
--- 
a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
+++ 
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
@@ -17,15 +17,29 @@
  */
 package org.apache.hadoop.hdds.scm.chillmode;
 
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
+
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.HddsConfigKeys;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
+import 
org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
 import org.apache.hadoop.hdds.scm.HddsTestUtils;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
+import 
org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
 import org.apache.hadoop.hdds.server.events.EventQueue;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.BeforeClass;
@@ -33,9 +47,6 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertFalse;
-
 /** Test class for SCMChillModeManager.
  */
 public class TestSCMChillModeManager {
@@ -65,7 +76,7 @@ public class TestSCMChillModeManager {
 
   @Test
   public void testChillModeStateWithNullContainers() {
-    new SCMChillModeManager(config, null, queue);
+    new SCMChillModeManager(config, null, null, queue);
   }
 
   private void testChillMode(int numContainers) throws Exception {
@@ -76,7 +87,8 @@ public class TestSCMChillModeManager {
     for (ContainerInfo container : containers) {
       container.setState(HddsProtos.LifeCycleState.OPEN);
     }
-    scmChillModeManager = new SCMChillModeManager(config, containers, queue);
+    scmChillModeManager = new SCMChillModeManager(
+        config, containers, null, queue);
     queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
         scmChillModeManager);
     assertTrue(scmChillModeManager.getInChillMode());
@@ -96,7 +108,8 @@ public class TestSCMChillModeManager {
     for (ContainerInfo container : containers) {
       container.setState(HddsProtos.LifeCycleState.CLOSED);
     }
-    scmChillModeManager = new SCMChillModeManager(config, containers, queue);
+    scmChillModeManager = new SCMChillModeManager(
+        config, containers, null, queue);
     queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
         scmChillModeManager);
     assertTrue(scmChillModeManager.getInChillMode());
@@ -118,7 +131,8 @@ public class TestSCMChillModeManager {
   public void testDisableChillMode() {
     OzoneConfiguration conf = new OzoneConfiguration(config);
     conf.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED, false);
-    scmChillModeManager = new SCMChillModeManager(conf, containers, queue);
+    scmChillModeManager = new SCMChillModeManager(
+        conf, containers, null, queue);
     assertFalse(scmChillModeManager.getInChillMode());
   }
 
@@ -149,7 +163,8 @@ public class TestSCMChillModeManager {
       container.setState(HddsProtos.LifeCycleState.OPEN);
     }
 
-    scmChillModeManager = new SCMChillModeManager(config, containers, queue);
+    scmChillModeManager = new SCMChillModeManager(
+        config, containers, null, queue);
     queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
         scmChillModeManager);
     assertTrue(scmChillModeManager.getInChillMode());
@@ -173,7 +188,8 @@ public class TestSCMChillModeManager {
   private void testChillModeDataNodes(int numOfDns) throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration(config);
     conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, numOfDns);
-    scmChillModeManager = new SCMChillModeManager(conf, containers, queue);
+    scmChillModeManager = new SCMChillModeManager(
+        conf, containers, null, queue);
     queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
         scmChillModeManager);
     // Assert SCM is in Chill mode.
@@ -212,4 +228,51 @@ public class TestSCMChillModeManager {
     }, 100, 2000 * 9);
   }
 
+  @Test
+  public void testChillModePipelineExitRule() throws Exception {
+    containers = new ArrayList<>();
+    containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
+    String storageDir = GenericTestUtils.getTempPath(
+        TestSCMChillModeManager.class.getName() + UUID.randomUUID());
+    try{
+      MockNodeManager nodeManager = new MockNodeManager(true, 1);
+      config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
+      // enable pipeline check
+      config.setBoolean(
+          HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+
+      PipelineManager pipelineManager = new SCMPipelineManager(config,
+          nodeManager, queue);
+      scmChillModeManager = new SCMChillModeManager(
+          config, containers, pipelineManager, queue);
+      queue.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+          scmChillModeManager);
+
+      queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
+          HddsTestUtils.createNodeRegistrationContainerReport(containers));
+      assertTrue(scmChillModeManager.getInChillMode());
+
+      // simulation a pipeline report to trigger the rule check
+      Pipeline pipeline = pipelineManager.createPipeline(
+          HddsProtos.ReplicationType.STAND_ALONE,
+          HddsProtos.ReplicationFactor.ONE);
+      PipelineReportsProto.Builder reportBuilder = PipelineReportsProto
+          .newBuilder();
+      reportBuilder.addPipelineReport(PipelineReport.newBuilder()
+          .setPipelineID(pipeline.getId().getProtobuf()));
+
+      queue.fireEvent(SCMEvents.PIPELINE_REPORT, new 
PipelineReportFromDatanode(
+          pipeline.getNodes().get(0), reportBuilder.build()));
+
+      GenericTestUtils.waitFor(() -> {
+        return !scmChillModeManager.getInChillMode();
+      }, 100, 1000 * 10);
+      pipelineManager.close();
+    } finally {
+      config.setBoolean(
+          HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
+          false);
+      FileUtil.fullyDelete(new File(storageDir));
+    }
+  }
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to