You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by bh...@apache.org on 2019/03/01 05:41:42 UTC
[hadoop] branch trunk updated: HDDS-1182. Pipeline Rule where at
least one datanode is reported in the pipeline.
This is an automated email from the ASF dual-hosted git repository.
bharat pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new 77b23c8 HDDS-1182. Pipeline Rule where at least one datanode is reported in the pipeline.
77b23c8 is described below
commit 77b23c816e5a4e29cbaa7bc4feb944264ec584f1
Author: Bharat Viswanadham <bh...@apache.org>
AuthorDate: Thu Feb 28 21:41:36 2019 -0800
HDDS-1182. Pipeline Rule where at least one datanode is reported in the pipeline.
---
.../org/apache/hadoop/hdds/HddsConfigKeys.java | 5 +
.../common/src/main/resources/ozone-default.xml | 10 +
.../chillmode/OneReplicaPipelineChillModeRule.java | 142 +++++++++++++++
.../hdds/scm/chillmode/SCMChillModeManager.java | 14 ++
.../TestOneReplicaPipelineChillModeRule.java | 202 +++++++++++++++++++++
.../hadoop/hdds/scm/chillmode/package-info.java | 21 +++
6 files changed, 394 insertions(+)
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 007104f..c265771 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
@@ -89,6 +89,11 @@ public final class HddsConfigKeys {
public static final double
HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10;
+ public static final String HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT =
+ "hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct";
+ public static final double
+ HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT = 0.90;
+
public static final String HDDS_LOCK_MAX_CONCURRENCY =
"hdds.lock.max.concurrency";
public static final int HDDS_LOCK_MAX_CONCURRENCY_DEFAULT = 100;
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index b114daa..faf2f89 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1326,6 +1326,16 @@
</property>
<property>
+ <name>hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct</name>
+ <value>0.90</value>
+ <tag>HDDS,SCM,OPERATION</tag>
+ <description>
+ Percentage of pipelines, where at least one datanode is reported in the
+ pipeline.
+ </description>
+ </property>
+
+ <property>
<name>hdds.container.action.max.limit</name>
<value>20</value>
<tag>DATANODE</tag>
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java
new file mode 100644
index 0000000..20b35b8
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java
@@ -0,0 +1,142 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.hdds.scm.chillmode;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.HddsConfigKeys;
+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.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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * This rule covers whether we have atleast one datanode is reported for each
+ * pipeline. This rule is for all open containers, we have at least one
+ * replica available for read when we exit chill mode.
+ */
+public class OneReplicaPipelineChillModeRule implements
+ ChillModeExitRule<PipelineReportFromDatanode>,
+ EventHandler<PipelineReportFromDatanode> {
+
+ private static final Logger LOG =
+ LoggerFactory.getLogger(OneReplicaPipelineChillModeRule.class);
+
+ private int thresholdCount;
+ private Set<PipelineID> reportedPipelineIDSet = new HashSet<>();
+ private final PipelineManager pipelineManager;
+ private final SCMChillModeManager chillModeManager;
+
+ public OneReplicaPipelineChillModeRule(PipelineManager pipelineManager,
+ SCMChillModeManager chillModeManager,
+ Configuration configuration) {
+ this.chillModeManager = chillModeManager;
+ this.pipelineManager = pipelineManager;
+
+ double percent =
+ configuration.getDouble(
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT,
+ HddsConfigKeys.
+ HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT);
+
+ int totalPipelineCount =
+ pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
+ HddsProtos.ReplicationFactor.THREE).size();
+
+ thresholdCount = (int) Math.ceil(percent * totalPipelineCount);
+
+ LOG.info(" Total pipeline count is {}, pipeline's with atleast one " +
+ "datanode reported threshold count is {}", totalPipelineCount,
+ thresholdCount);
+
+ }
+ @Override
+ public boolean validate() {
+ if (reportedPipelineIDSet.size() >= thresholdCount) {
+ return true;
+ }
+ return false;
+ }
+
+ @Override
+ public void process(PipelineReportFromDatanode pipelineReportFromDatanode) {
+ 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.getFactor() == HddsProtos.ReplicationFactor.THREE &&
+ !reportedPipelineIDSet.contains(pipelineID)) {
+ reportedPipelineIDSet.add(pipelineID);
+ }
+ }
+ }
+
+ @Override
+ public void cleanup() {
+ reportedPipelineIDSet.clear();
+ }
+
+ @Override
+ public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
+ EventPublisher publisher) {
+
+ if (validate()) {
+ chillModeManager.validateChillModeExitRules(publisher);
+ return;
+ }
+
+ // Process pipeline report from datanode
+ process(pipelineReportFromDatanode);
+
+ if (chillModeManager.getInChillMode()) {
+ SCMChillModeManager.getLogger().info(
+ "SCM in chill mode. Pipelines with atleast one datanode reported " +
+ "count is {}, required atleast one datanode reported per " +
+ "pipeline count is {}",
+ reportedPipelineIDSet.size(), thresholdCount);
+ }
+
+ if (validate()) {
+ chillModeManager.validateChillModeExitRules(publisher);
+ }
+ }
+}
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 ba79af7..1d97a57 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
@@ -62,6 +62,8 @@ public class SCMChillModeManager implements
private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
private static final String HEALTHY_PIPELINE_EXIT_RULE =
"HealthyPipelineChillModeRule";
+ private static final String ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE =
+ "AtleastOneDatanodeReportedRule";
private final EventQueue eventPublisher;
private final PipelineManager pipelineManager;
@@ -86,8 +88,14 @@ public class SCMChillModeManager implements
&& pipelineManager != null) {
HealthyPipelineChillModeRule rule = new HealthyPipelineChillModeRule(
pipelineManager, this, config);
+ OneReplicaPipelineChillModeRule oneReplicaPipelineChillModeRule =
+ new OneReplicaPipelineChillModeRule(pipelineManager, this, conf);
exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, rule);
+ exitRules.put(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE,
+ oneReplicaPipelineChillModeRule);
eventPublisher.addHandler(SCMEvents.PROCESSED_PIPELINE_REPORT, rule);
+ eventPublisher.addHandler(SCMEvents.PROCESSED_PIPELINE_REPORT,
+ oneReplicaPipelineChillModeRule);
}
emitChillModeStatus();
} else {
@@ -179,4 +187,10 @@ public class SCMChillModeManager implements
exitRules.get(HEALTHY_PIPELINE_EXIT_RULE);
}
+ @VisibleForTesting
+ public OneReplicaPipelineChillModeRule getOneReplicaPipelineChillModeRule() {
+ return (OneReplicaPipelineChillModeRule)
+ exitRules.get(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE);
+ }
+
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java
new file mode 100644
index 0000000..f389a83
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java
@@ -0,0 +1,202 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+
+package org.apache.hadoop.hdds.scm.chillmode;
+
+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;
+import org.apache.hadoop.hdds.server.events.EventQueue;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * This class tests OneReplicaPipelineChillModeRule.
+ */
+public class TestOneReplicaPipelineChillModeRule {
+
+ @Rule
+ public TemporaryFolder folder = new TemporaryFolder();
+ private OneReplicaPipelineChillModeRule rule;
+ private PipelineManager pipelineManager;
+ private EventQueue eventQueue;
+
+
+ private void setup(int nodes, int pipelineFactorThreeCount,
+ int pipelineFactorOneCount) throws Exception {
+ OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+ ozoneConfiguration.setBoolean(
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ ozoneConfiguration.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+ folder.newFolder().toString());
+
+ List<ContainerInfo> containers = new ArrayList<>();
+ containers.addAll(HddsTestUtils.getContainerInfo(1));
+ MockNodeManager mockNodeManager = new MockNodeManager(true, nodes);
+
+ eventQueue = new EventQueue();
+ pipelineManager =
+ new SCMPipelineManager(ozoneConfiguration, mockNodeManager,
+ eventQueue);
+
+ createPipelines(pipelineFactorThreeCount,
+ HddsProtos.ReplicationFactor.THREE);
+ createPipelines(pipelineFactorOneCount,
+ HddsProtos.ReplicationFactor.ONE);
+
+ SCMChillModeManager scmChillModeManager =
+ new SCMChillModeManager(ozoneConfiguration, containers,
+ pipelineManager, eventQueue);
+
+ rule = scmChillModeManager.getOneReplicaPipelineChillModeRule();
+ }
+
+ @Test
+ public void testOneReplicaPipelineRule() throws Exception {
+
+ // As with 30 nodes, We can create 7 pipelines with replication factor 3.
+ // (This is because in node manager for every 10 nodes, 7 nodes are
+ // healthy, 2 are stale one is dead.)
+ int nodes = 30;
+ int pipelineFactorThreeCount = 7;
+ int pipelineCountOne = 0;
+ setup(nodes, pipelineFactorThreeCount, pipelineCountOne);
+
+ GenericTestUtils.LogCapturer logCapturer =
+ GenericTestUtils.LogCapturer.captureLogs(
+ LoggerFactory.getLogger(SCMChillModeManager.class));
+
+ List<Pipeline> pipelines = pipelineManager.getPipelines();
+ for (int i = 0; i < pipelineFactorThreeCount -1; i++) {
+ firePipelineEvent(pipelines.get(i));
+ }
+
+ // As 90% of 7 with ceil is 7, if we send 6 pipeline reports, rule
+ // validate should be still false.
+
+ GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
+ "reported count is 6"), 1000, 5000);
+
+ Assert.assertFalse(rule.validate());
+
+ //Fire last pipeline event from datanode.
+ firePipelineEvent(pipelines.get(pipelineFactorThreeCount - 1));
+
+ GenericTestUtils.waitFor(() -> rule.validate(), 1000, 5000);
+
+ }
+
+
+ @Test
+ public void testOneReplicaPipelineRuleMixedPipelines() throws Exception {
+
+ // As with 30 nodes, We can create 7 pipelines with replication factor 3.
+ // (This is because in node manager for every 10 nodes, 7 nodes are
+ // healthy, 2 are stale one is dead.)
+ int nodes = 30;
+ int pipelineCountThree = 7;
+ int pipelineCountOne = 21;
+
+ setup(nodes, pipelineCountThree, pipelineCountOne);
+
+ GenericTestUtils.LogCapturer logCapturer =
+ GenericTestUtils.LogCapturer.captureLogs(
+ LoggerFactory.getLogger(SCMChillModeManager.class));
+
+ List<Pipeline> pipelines =
+ pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
+ HddsProtos.ReplicationFactor.ONE);
+ for (int i = 0; i < pipelineCountOne; i++) {
+ firePipelineEvent(pipelines.get(i));
+ }
+
+ GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
+ "reported count is 0"), 1000, 5000);
+
+ // fired events for one node ratis pipeline, so we will be still false.
+ Assert.assertFalse(rule.validate());
+
+ pipelines =
+ pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
+ HddsProtos.ReplicationFactor.THREE);
+ for (int i = 0; i < pipelineCountThree - 1; i++) {
+ firePipelineEvent(pipelines.get(i));
+ }
+
+ GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
+ "reported count is 6"), 1000, 5000);
+
+ //Fire last pipeline event from datanode.
+ firePipelineEvent(pipelines.get(pipelineCountThree - 1));
+
+ GenericTestUtils.waitFor(() -> rule.validate(), 1000, 5000);
+
+ }
+
+
+
+ private void createPipelines(int count,
+ HddsProtos.ReplicationFactor factor) throws Exception {
+ for (int i = 0; i < count; i++) {
+ pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
+ factor);
+ }
+ }
+
+ private void firePipelineEvent(Pipeline pipeline) {
+ PipelineReportsProto.Builder reportBuilder =
+ PipelineReportsProto.newBuilder();
+
+ reportBuilder.addPipelineReport(PipelineReport.newBuilder()
+ .setPipelineID(pipeline.getId().getProtobuf()));
+
+ if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE) {
+ eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
+ new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode(
+ pipeline.getNodes().get(0), reportBuilder.build()));
+ eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
+ new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode(
+ pipeline.getNodes().get(1), reportBuilder.build()));
+ eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
+ new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode(
+ pipeline.getNodes().get(2), reportBuilder.build()));
+ } else {
+ eventQueue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
+ new SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode(
+ pipeline.getNodes().get(0), reportBuilder.build()));
+ }
+ }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
new file mode 100644
index 0000000..22a522a
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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;
+/**
+ * SCM Chill mode tests.
+ */
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org