Skip to content

Commit db0361c

Browse files
Fix test failure and added little more changes
1 parent bebe004 commit db0361c

File tree

3 files changed

+19
-1
lines changed

3 files changed

+19
-1
lines changed

hadoop-hdds/common/src/main/resources/ozone-default.xml

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1315,6 +1315,16 @@
13151315
</description>
13161316
</property>
13171317

1318+
<property>
1319+
<name>hdds.scm.chillmode.healthy.pipelie.pct</name>
1320+
<value>0.10</value>
1321+
<tag>HDDS,SCM,OPERATION</tag>
1322+
<description>
1323+
Percentage of healthy pipelines, where all 3 datanodes are reported in the
1324+
pipeline.
1325+
</description>
1326+
</property>
1327+
13181328
<property>
13191329
<name>hdds.container.action.max.limit</name>
13201330
<value>20</value>

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/HealthyPipelineChillModeRule.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
import org.apache.hadoop.conf.Configuration;
2121
import org.apache.hadoop.hdds.HddsConfigKeys;
22+
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
2223
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
2324
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
2425
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -61,7 +62,10 @@ public class HealthyPipelineChillModeRule
6162
HddsConfigKeys.
6263
HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT);
6364

64-
int pipelineCount = pipelineManager.getPipelines().size();
65+
// As we want to wait for 3 node pipelines
66+
int pipelineCount =
67+
pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
68+
HddsProtos.ReplicationFactor.THREE).size();
6569

6670
// This value will be zero when pipeline count is 0.
6771
// On a fresh installed cluster, there will be zero pipelines in the SCM

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,10 @@ public final class SCMEvents {
103103
public static final TypedEvent<PipelineReportFromDatanode> PIPELINE_REPORT =
104104
new TypedEvent<>(PipelineReportFromDatanode.class, "Pipeline_Report");
105105

106+
/**
107+
* PipelineReport processed by pipeline report handler. This event is
108+
* received by HealthyPipelineChillModeRule.
109+
*/
106110
public static final TypedEvent<PipelineReportFromDatanode>
107111
PROCESSED_PIPELINE_REPORT = new TypedEvent<>(
108112
PipelineReportFromDatanode.class, "Processed_Pipeline_Report");

0 commit comments

Comments
 (0)