Skip to content

Commit befd64e

Browse files
HDDS-11694. Safemode Improvement: Introduce factory class to create safemode rules. (#7433)
1 parent a46153d commit befd64e

8 files changed

Lines changed: 290 additions & 47 deletions

File tree

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,12 @@ public class ContainerSafeModeRule extends
6868
private AtomicLong ecContainerWithMinReplicas = new AtomicLong(0);
6969
private final ContainerManager containerManager;
7070

71+
public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
72+
ConfigurationSource conf,
73+
ContainerManager containerManager, SCMSafeModeManager manager) {
74+
this(ruleName, eventQueue, conf, containerManager.getContainers(), containerManager, manager);
75+
}
76+
7177
public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
7278
ConfigurationSource conf,
7379
List<ContainerInfo> containers,

hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java

Lines changed: 15 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import java.util.Map;
2424
import java.util.Set;
2525
import java.util.concurrent.atomic.AtomicBoolean;
26+
import java.util.stream.Collectors;
2627

2728
import org.apache.commons.lang3.tuple.Pair;
2829
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -90,7 +91,7 @@ public class SCMSafeModeManager implements SafeModeManager {
9091
private AtomicBoolean preCheckComplete = new AtomicBoolean(false);
9192
private AtomicBoolean forceExitSafeMode = new AtomicBoolean(false);
9293

93-
private Map<String, SafeModeExitRule> exitRules = new HashMap(1);
94+
private Map<String, SafeModeExitRule> exitRules = new HashMap<>(1);
9495
private Set<String> preCheckRules = new HashSet<>(1);
9596
private ConfigurationSource config;
9697
private static final String CONT_EXIT_RULE = "ContainerSafeModeRule";
@@ -110,6 +111,8 @@ public class SCMSafeModeManager implements SafeModeManager {
110111

111112
private final SafeModeMetrics safeModeMetrics;
112113

114+
115+
// TODO: Remove allContainers argument. (HDDS-11795)
113116
public SCMSafeModeManager(ConfigurationSource conf,
114117
List<ContainerInfo> allContainers,
115118
ContainerManager containerManager, PipelineManager pipelineManager,
@@ -126,30 +129,17 @@ public SCMSafeModeManager(ConfigurationSource conf,
126129

127130
if (isSafeModeEnabled) {
128131
this.safeModeMetrics = SafeModeMetrics.create();
129-
ContainerSafeModeRule containerSafeModeRule =
130-
new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
131-
allContainers, containerManager, this);
132-
DataNodeSafeModeRule dataNodeSafeModeRule =
133-
new DataNodeSafeModeRule(DN_EXIT_RULE, eventQueue, config, this);
134-
exitRules.put(CONT_EXIT_RULE, containerSafeModeRule);
135-
exitRules.put(DN_EXIT_RULE, dataNodeSafeModeRule);
136-
preCheckRules.add(DN_EXIT_RULE);
137-
if (conf.getBoolean(
138-
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
139-
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
140-
&& pipelineManager != null) {
141-
HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
142-
new HealthyPipelineSafeModeRule(HEALTHY_PIPELINE_EXIT_RULE,
143-
eventQueue, pipelineManager,
144-
this, config, scmContext);
145-
OneReplicaPipelineSafeModeRule oneReplicaPipelineSafeModeRule =
146-
new OneReplicaPipelineSafeModeRule(
147-
ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE, eventQueue,
148-
pipelineManager, this, conf);
149-
exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, healthyPipelineSafeModeRule);
150-
exitRules.put(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE,
151-
oneReplicaPipelineSafeModeRule);
152-
}
132+
133+
// TODO: Remove the cyclic ("this") dependency (HDDS-11797)
134+
SafeModeRuleFactory.initialize(config, scmContext, eventQueue,
135+
this, pipelineManager, containerManager);
136+
SafeModeRuleFactory factory = SafeModeRuleFactory.getInstance();
137+
138+
exitRules = factory.getSafeModeRules().stream().collect(
139+
Collectors.toMap(SafeModeExitRule::getRuleName, rule -> rule));
140+
141+
preCheckRules = factory.getPreCheckRules().stream()
142+
.map(SafeModeExitRule::getRuleName).collect(Collectors.toSet());
153143
} else {
154144
this.safeModeMetrics = null;
155145
exitSafeMode(eventQueue, true);
Lines changed: 131 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,131 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements.  See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership.  The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License.  You may obtain a copy of the License at
9+
*
10+
*      http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hadoop.hdds.scm.safemode;
20+
21+
22+
import org.apache.hadoop.hdds.HddsConfigKeys;
23+
import org.apache.hadoop.hdds.conf.ConfigurationSource;
24+
import org.apache.hadoop.hdds.scm.container.ContainerManager;
25+
import org.apache.hadoop.hdds.scm.ha.SCMContext;
26+
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
27+
import org.apache.hadoop.hdds.server.events.EventQueue;
28+
import org.slf4j.Logger;
29+
import org.slf4j.LoggerFactory;
30+
31+
import java.util.ArrayList;
32+
import java.util.List;
33+
34+
/**
35+
* Factory to create SafeMode rules.
36+
*/
37+
public final class SafeModeRuleFactory {
38+
39+
40+
private static final Logger LOG = LoggerFactory.getLogger(SafeModeRuleFactory.class);
41+
42+
// TODO: Move the rule names to respective rules. (HDDS-11798)
43+
private static final String CONT_EXIT_RULE = "ContainerSafeModeRule";
44+
private static final String DN_EXIT_RULE = "DataNodeSafeModeRule";
45+
private static final String HEALTHY_PIPELINE_EXIT_RULE =
46+
"HealthyPipelineSafeModeRule";
47+
private static final String ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE =
48+
"AtleastOneDatanodeReportedRule";
49+
50+
private final ConfigurationSource config;
51+
private final SCMContext scmContext;
52+
private final EventQueue eventQueue;
53+
54+
// TODO: Remove dependency on safeModeManager (HDDS-11797)
55+
private final SCMSafeModeManager safeModeManager;
56+
private final PipelineManager pipelineManager;
57+
private final ContainerManager containerManager;
58+
59+
private final List<SafeModeExitRule<?>> safeModeRules;
60+
private final List<SafeModeExitRule<?>> preCheckRules;
61+
62+
private static SafeModeRuleFactory instance;
63+
64+
private SafeModeRuleFactory(final ConfigurationSource config,
65+
final SCMContext scmContext,
66+
final EventQueue eventQueue,
67+
final SCMSafeModeManager safeModeManager,
68+
final PipelineManager pipelineManager,
69+
final ContainerManager containerManager) {
70+
this.config = config;
71+
this.scmContext = scmContext;
72+
this.eventQueue = eventQueue;
73+
this.safeModeManager = safeModeManager;
74+
this.pipelineManager = pipelineManager;
75+
this.containerManager = containerManager;
76+
this.safeModeRules = new ArrayList<>();
77+
this.preCheckRules = new ArrayList<>();
78+
loadRules();
79+
}
80+
81+
private void loadRules() {
82+
// TODO: Use annotation to load the rules. (HDDS-11730)
83+
safeModeRules.add(new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
84+
containerManager, safeModeManager));
85+
SafeModeExitRule<?> dnRule = new DataNodeSafeModeRule(DN_EXIT_RULE, eventQueue, config, safeModeManager);
86+
safeModeRules.add(dnRule);
87+
preCheckRules.add(dnRule);
88+
89+
// TODO: Move isRuleEnabled check to the Rule implementation. (HDDS-11799)
90+
if (config.getBoolean(
91+
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
92+
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
93+
&& pipelineManager != null) {
94+
95+
safeModeRules.add(new HealthyPipelineSafeModeRule(HEALTHY_PIPELINE_EXIT_RULE,
96+
eventQueue, pipelineManager, safeModeManager, config, scmContext));
97+
safeModeRules.add(new OneReplicaPipelineSafeModeRule(
98+
ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE, eventQueue,
99+
pipelineManager, safeModeManager, config));
100+
}
101+
102+
}
103+
104+
public static synchronized SafeModeRuleFactory getInstance() {
105+
if (instance != null) {
106+
return instance;
107+
}
108+
throw new IllegalStateException("SafeModeRuleFactory not initialized," +
109+
" call initialize method before getInstance.");
110+
}
111+
112+
// TODO: Refactor and reduce the arguments. (HDDS-11800)
113+
public static synchronized void initialize(
114+
final ConfigurationSource config,
115+
final SCMContext scmContext,
116+
final EventQueue eventQueue,
117+
final SCMSafeModeManager safeModeManager,
118+
final PipelineManager pipelineManager,
119+
final ContainerManager containerManager) {
120+
instance = new SafeModeRuleFactory(config, scmContext, eventQueue,
121+
safeModeManager, pipelineManager, containerManager);
122+
}
123+
124+
public List<SafeModeExitRule<?>> getSafeModeRules() {
125+
return safeModeRules;
126+
}
127+
128+
public List<SafeModeExitRule<?>> getPreCheckRules() {
129+
return preCheckRules;
130+
}
131+
}

hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineManagerImpl.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -358,7 +358,8 @@ public void testClosePipelineShouldFailOnFollower() throws Exception {
358358
public void testPipelineReport() throws Exception {
359359
try (PipelineManagerImpl pipelineManager = createPipelineManager(true)) {
360360
SCMSafeModeManager scmSafeModeManager =
361-
new SCMSafeModeManager(conf, new ArrayList<>(), null, pipelineManager,
361+
new SCMSafeModeManager(conf, new ArrayList<>(),
362+
mock(ContainerManager.class), pipelineManager,
362363
new EventQueue(), serviceManager, scmContext);
363364
Pipeline pipeline = pipelineManager
364365
.createPipeline(RatisReplicationConfig
@@ -469,7 +470,7 @@ public void testPipelineOpenOnlyWhenLeaderReported() throws Exception {
469470

470471
SCMSafeModeManager scmSafeModeManager =
471472
new SCMSafeModeManager(new OzoneConfiguration(), new ArrayList<>(),
472-
null, pipelineManager, new EventQueue(),
473+
mock(ContainerManager.class), pipelineManager, new EventQueue(),
473474
serviceManager, scmContext);
474475
PipelineReportHandler pipelineReportHandler =
475476
new PipelineReportHandler(scmSafeModeManager, pipelineManager,

hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@
3131
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
3232
import org.apache.hadoop.hdds.scm.HddsTestUtils;
3333
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
34+
import org.apache.hadoop.hdds.scm.container.ContainerManager;
3435
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
3536
import org.apache.hadoop.hdds.scm.events.SCMEvents;
3637
import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
@@ -50,6 +51,8 @@
5051

5152
import static org.junit.jupiter.api.Assertions.assertTrue;
5253
import static org.junit.jupiter.api.Assertions.assertFalse;
54+
import static org.mockito.Mockito.mock;
55+
import static org.mockito.Mockito.when;
5356

5457
/**
5558
* This class tests HealthyPipelineSafeMode rule.
@@ -69,6 +72,8 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines()
6972

7073
OzoneConfiguration config = new OzoneConfiguration();
7174
MockNodeManager nodeManager = new MockNodeManager(true, 0);
75+
ContainerManager containerManager = mock(ContainerManager.class);
76+
when(containerManager.getContainers()).thenReturn(containers);
7277
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
7378
// enable pipeline check
7479
config.setBoolean(
@@ -94,7 +99,7 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines()
9499
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
95100
mockRatisProvider);
96101
SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
97-
config, containers, null, pipelineManager, eventQueue,
102+
config, containers, containerManager, pipelineManager, eventQueue,
98103
serviceManager, scmContext);
99104

100105
HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
@@ -121,6 +126,8 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception {
121126
// stale and last one is dead, and this repeats. So for a 12 node, 9
122127
// healthy, 2 stale and one dead.
123128
MockNodeManager nodeManager = new MockNodeManager(true, 12);
129+
ContainerManager containerManager = mock(ContainerManager.class);
130+
when(containerManager.getContainers()).thenReturn(containers);
124131
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
125132
// enable pipeline check
126133
config.setBoolean(
@@ -172,7 +179,7 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception {
172179
MockRatisPipelineProvider.markPipelineHealthy(pipeline3);
173180

174181
SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
175-
config, containers, null, pipelineManager, eventQueue,
182+
config, containers, containerManager, pipelineManager, eventQueue,
176183
serviceManager, scmContext);
177184

178185
HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
@@ -215,6 +222,8 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines()
215222
// stale and last one is dead, and this repeats. So for a 12 node, 9
216223
// healthy, 2 stale and one dead.
217224
MockNodeManager nodeManager = new MockNodeManager(true, 12);
225+
ContainerManager containerManager = mock(ContainerManager.class);
226+
when(containerManager.getContainers()).thenReturn(containers);
218227
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
219228
// enable pipeline check
220229
config.setBoolean(
@@ -266,7 +275,7 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines()
266275
MockRatisPipelineProvider.markPipelineHealthy(pipeline3);
267276

268277
SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
269-
config, containers, null, pipelineManager, eventQueue,
278+
config, containers, containerManager, pipelineManager, eventQueue,
270279
serviceManager, scmContext);
271280

272281
HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =

hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
3636
import org.apache.hadoop.hdds.scm.HddsTestUtils;
3737
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
38+
import org.apache.hadoop.hdds.scm.container.ContainerManager;
3839
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
3940
import org.apache.hadoop.hdds.scm.events.SCMEvents;
4041
import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
@@ -58,6 +59,8 @@
5859
import org.slf4j.LoggerFactory;
5960

6061
import static org.junit.jupiter.api.Assertions.assertFalse;
62+
import static org.mockito.Mockito.mock;
63+
import static org.mockito.Mockito.when;
6164

6265
/**
6366
* This class tests OneReplicaPipelineSafeModeRule.
@@ -86,7 +89,8 @@ private void setup(int nodes, int pipelineFactorThreeCount,
8689
List<ContainerInfo> containers = new ArrayList<>();
8790
containers.addAll(HddsTestUtils.getContainerInfo(1));
8891
mockNodeManager = new MockNodeManager(true, nodes);
89-
92+
ContainerManager containerManager = mock(ContainerManager.class);
93+
when(containerManager.getContainers()).thenReturn(containers);
9094
eventQueue = new EventQueue();
9195
serviceManager = new SCMServiceManager();
9296
scmContext = SCMContext.emptyContext();
@@ -116,7 +120,7 @@ private void setup(int nodes, int pipelineFactorThreeCount,
116120
HddsProtos.ReplicationFactor.ONE);
117121

118122
SCMSafeModeManager scmSafeModeManager =
119-
new SCMSafeModeManager(ozoneConfiguration, containers, null,
123+
new SCMSafeModeManager(ozoneConfiguration, containers, containerManager,
120124
pipelineManager, eventQueue, serviceManager, scmContext);
121125

122126
rule = scmSafeModeManager.getOneReplicaPipelineSafeModeRule();

0 commit comments

Comments
 (0)