|
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.chillmode;
|
|
|
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
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;
|
|
@@ -34,6 +35,9 @@ import com.google.common.base.Preconditions;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.Set;
|
|
|
+
|
|
|
/**
|
|
|
* Class defining Chill mode exit criteria for Pipelines.
|
|
|
*
|
|
@@ -45,12 +49,14 @@ public class HealthyPipelineChillModeRule
|
|
|
implements ChillModeExitRule<PipelineReportFromDatanode>,
|
|
|
EventHandler<PipelineReportFromDatanode> {
|
|
|
|
|
|
- private static final Logger LOG =
|
|
|
+ public static final Logger LOG =
|
|
|
LoggerFactory.getLogger(HealthyPipelineChillModeRule.class);
|
|
|
private final PipelineManager pipelineManager;
|
|
|
private final SCMChillModeManager chillModeManager;
|
|
|
private final int healthyPipelineThresholdCount;
|
|
|
private int currentHealthyPipelineCount = 0;
|
|
|
+ private final Set<DatanodeDetails> processedDatanodeDetails =
|
|
|
+ new HashSet<>();
|
|
|
|
|
|
HealthyPipelineChillModeRule(PipelineManager pipelineManager,
|
|
|
SCMChillModeManager manager, Configuration configuration) {
|
|
@@ -71,7 +77,7 @@ public class HealthyPipelineChillModeRule
|
|
|
// On a fresh installed cluster, there will be zero pipelines in the SCM
|
|
|
// pipeline DB.
|
|
|
healthyPipelineThresholdCount =
|
|
|
- (int) Math.ceil((healthyPipelinesPercent / 100) * pipelineCount);
|
|
|
+ (int) Math.ceil(healthyPipelinesPercent * pipelineCount);
|
|
|
|
|
|
LOG.info(" Total pipeline count is {}, healthy pipeline " +
|
|
|
"threshold count is {}", pipelineCount, healthyPipelineThresholdCount);
|
|
@@ -101,7 +107,8 @@ public class HealthyPipelineChillModeRule
|
|
|
continue;
|
|
|
}
|
|
|
|
|
|
- if (pipeline.getPipelineState() == Pipeline.PipelineState.OPEN) {
|
|
|
+ if (pipeline.getFactor() == HddsProtos.ReplicationFactor.THREE &&
|
|
|
+ pipeline.getPipelineState() == Pipeline.PipelineState.OPEN) {
|
|
|
// If the pipeline is open state mean, all 3 datanodes are reported
|
|
|
// for this pipeline.
|
|
|
currentHealthyPipelineCount++;
|
|
@@ -125,14 +132,26 @@ public class HealthyPipelineChillModeRule
|
|
|
return;
|
|
|
}
|
|
|
|
|
|
- // Process pipeline report from datanode
|
|
|
- process(pipelineReportFromDatanode);
|
|
|
|
|
|
- if (chillModeManager.getInChillMode()) {
|
|
|
- SCMChillModeManager.getLogger().info(
|
|
|
- "SCM in chill mode. Healthy pipelines reported count is {}, " +
|
|
|
- "required healthy pipeline reported count is {}",
|
|
|
- currentHealthyPipelineCount, healthyPipelineThresholdCount);
|
|
|
+ // When SCM is in chill mode for long time, already registered
|
|
|
+ // datanode can send pipeline report again, then pipeline handler fires
|
|
|
+ // processed report event, we should not consider this pipeline report
|
|
|
+ // from datanode again during threshold calculation.
|
|
|
+ DatanodeDetails dnDetails = pipelineReportFromDatanode.getDatanodeDetails();
|
|
|
+ if (!processedDatanodeDetails.contains(
|
|
|
+ pipelineReportFromDatanode.getDatanodeDetails())) {
|
|
|
+
|
|
|
+ // Process pipeline report from datanode
|
|
|
+ process(pipelineReportFromDatanode);
|
|
|
+
|
|
|
+ if (chillModeManager.getInChillMode()) {
|
|
|
+ SCMChillModeManager.getLogger().info(
|
|
|
+ "SCM in chill mode. Healthy pipelines reported count is {}, " +
|
|
|
+ "required healthy pipeline reported count is {}",
|
|
|
+ currentHealthyPipelineCount, healthyPipelineThresholdCount);
|
|
|
+ }
|
|
|
+
|
|
|
+ processedDatanodeDetails.add(dnDetails);
|
|
|
}
|
|
|
|
|
|
if (validate()) {
|