Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-11694. Safemode Improvement: Introduce factory class to create safemode rules. #7433

Merged
merged 6 commits into from
Dec 9, 2024
Merged
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,12 @@ public class ContainerSafeModeRule extends
private AtomicLong ecContainerWithMinReplicas = new AtomicLong(0);
private final ContainerManager containerManager;

public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
ConfigurationSource conf,
ContainerManager containerManager, SCMSafeModeManager manager) {
this(ruleName, eventQueue, conf, containerManager.getContainers(), containerManager, manager);
}

public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
ConfigurationSource conf,
List<ContainerInfo> containers,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.stream.Collectors;

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

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

private final SafeModeMetrics safeModeMetrics;


// TODO: Remove allContainers argument. (HDDS-11795)
public SCMSafeModeManager(ConfigurationSource conf,
List<ContainerInfo> allContainers,
ContainerManager containerManager, PipelineManager pipelineManager,
Expand All @@ -126,30 +129,17 @@ public SCMSafeModeManager(ConfigurationSource conf,

if (isSafeModeEnabled) {
this.safeModeMetrics = SafeModeMetrics.create();
ContainerSafeModeRule containerSafeModeRule =
new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
allContainers, containerManager, this);
DataNodeSafeModeRule dataNodeSafeModeRule =
new DataNodeSafeModeRule(DN_EXIT_RULE, eventQueue, config, this);
exitRules.put(CONT_EXIT_RULE, containerSafeModeRule);
exitRules.put(DN_EXIT_RULE, dataNodeSafeModeRule);
preCheckRules.add(DN_EXIT_RULE);
if (conf.getBoolean(
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
&& pipelineManager != null) {
HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
new HealthyPipelineSafeModeRule(HEALTHY_PIPELINE_EXIT_RULE,
eventQueue, pipelineManager,
this, config, scmContext);
OneReplicaPipelineSafeModeRule oneReplicaPipelineSafeModeRule =
new OneReplicaPipelineSafeModeRule(
ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE, eventQueue,
pipelineManager, this, conf);
exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, healthyPipelineSafeModeRule);
exitRules.put(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE,
oneReplicaPipelineSafeModeRule);
}

// TODO: Remove the cyclic ("this") dependency (HDDS-11797)
SafeModeRuleFactory.initialize(config, scmContext, eventQueue,
this, pipelineManager, containerManager);
SafeModeRuleFactory factory = SafeModeRuleFactory.getInstance();

exitRules = factory.getSafeModeRules().stream().collect(
Collectors.toMap(SafeModeExitRule::getRuleName, rule -> rule));

preCheckRules = factory.getPreCheckRules().stream()
.map(SafeModeExitRule::getRuleName).collect(Collectors.toSet());
} else {
this.safeModeMetrics = null;
exitSafeMode(eventQueue, true);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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.safemode;


import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.ha.SCMContext;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.ArrayList;
import java.util.List;

/**
* Factory to create SafeMode rules.
*/
public final class SafeModeRuleFactory {


private static final Logger LOG = LoggerFactory.getLogger(SafeModeRuleFactory.class);

// TODO: Move the rule names to respective rules. (HDDS-11798)
private static final String CONT_EXIT_RULE = "ContainerSafeModeRule";
private static final String DN_EXIT_RULE = "DataNodeSafeModeRule";
private static final String HEALTHY_PIPELINE_EXIT_RULE =
"HealthyPipelineSafeModeRule";
private static final String ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE =
"AtleastOneDatanodeReportedRule";

private final ConfigurationSource config;
private final SCMContext scmContext;
private final EventQueue eventQueue;

// TODO: Remove dependency on safeModeManager (HDDS-11797)
private final SCMSafeModeManager safeModeManager;
private final PipelineManager pipelineManager;
private final ContainerManager containerManager;

private final List<SafeModeExitRule<?>> safeModeRules;
private final List<SafeModeExitRule<?>> preCheckRules;

private static SafeModeRuleFactory instance;

private SafeModeRuleFactory(final ConfigurationSource config,
final SCMContext scmContext,
final EventQueue eventQueue,
final SCMSafeModeManager safeModeManager,
final PipelineManager pipelineManager,
final ContainerManager containerManager) {
this.config = config;
this.scmContext = scmContext;
this.eventQueue = eventQueue;
this.safeModeManager = safeModeManager;
this.pipelineManager = pipelineManager;
this.containerManager = containerManager;
this.safeModeRules = new ArrayList<>();
this.preCheckRules = new ArrayList<>();
loadRules();
}

private void loadRules() {
// TODO: Use annotation to load the rules. (HDDS-11730)
safeModeRules.add(new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
containerManager, safeModeManager));
SafeModeExitRule<?> dnRule = new DataNodeSafeModeRule(DN_EXIT_RULE, eventQueue, config, safeModeManager);
safeModeRules.add(dnRule);
preCheckRules.add(dnRule);

// TODO: Move isRuleEnabled check to the Rule implementation. (HDDS-11799)
if (config.getBoolean(
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
&& pipelineManager != null) {

safeModeRules.add(new HealthyPipelineSafeModeRule(HEALTHY_PIPELINE_EXIT_RULE,
eventQueue, pipelineManager, safeModeManager, config, scmContext));
safeModeRules.add(new OneReplicaPipelineSafeModeRule(
ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE, eventQueue,
pipelineManager, safeModeManager, config));
}

}

public static SafeModeRuleFactory getInstance() {
if (instance != null) {
return instance;
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

For thread-safety,

  • getInstance() should be synchronized, OR
  • instance should be volatile, and assigned to local variable first

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same suggestion.

Except this, others looks good to me.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not sure how I missed adding synchronized here. Thanks for the catch @adoroszlai & @ChenSammi.

throw new IllegalStateException("SafeModeRuleFactory not initialized," +
" call initialize method before getInstance.");
}

// TODO: Refactor and reduce the arguments. (HDDS-11800)
public static synchronized void initialize(
final ConfigurationSource config,
final SCMContext scmContext,
final EventQueue eventQueue,
final SCMSafeModeManager safeModeManager,
final PipelineManager pipelineManager,
final ContainerManager containerManager) {
instance = new SafeModeRuleFactory(config, scmContext, eventQueue,
safeModeManager, pipelineManager, containerManager);
}

public List<SafeModeExitRule<?>> getSafeModeRules() {
return safeModeRules;
}

public List<SafeModeExitRule<?>> getPreCheckRules() {
return preCheckRules;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,7 @@
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.mockito.ArgumentCaptor;
import org.mockito.Mockito;
import org.slf4j.LoggerFactory;

import java.io.File;
Expand Down Expand Up @@ -358,7 +359,8 @@ public void testClosePipelineShouldFailOnFollower() throws Exception {
public void testPipelineReport() throws Exception {
try (PipelineManagerImpl pipelineManager = createPipelineManager(true)) {
SCMSafeModeManager scmSafeModeManager =
new SCMSafeModeManager(conf, new ArrayList<>(), null, pipelineManager,
new SCMSafeModeManager(conf, new ArrayList<>(),
Mockito.mock(ContainerManager.class), pipelineManager,
new EventQueue(), serviceManager, scmContext);
Pipeline pipeline = pipelineManager
.createPipeline(RatisReplicationConfig
Expand Down Expand Up @@ -469,7 +471,7 @@ public void testPipelineOpenOnlyWhenLeaderReported() throws Exception {

SCMSafeModeManager scmSafeModeManager =
new SCMSafeModeManager(new OzoneConfiguration(), new ArrayList<>(),
null, pipelineManager, new EventQueue(),
Mockito.mock(ContainerManager.class), pipelineManager, new EventQueue(),
serviceManager, scmContext);
PipelineReportHandler pipelineReportHandler =
new PipelineReportHandler(scmSafeModeManager, pipelineManager,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
Expand All @@ -46,6 +47,7 @@
import org.apache.ozone.test.GenericTestUtils;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.mockito.Mockito;
import org.slf4j.LoggerFactory;

import static org.junit.jupiter.api.Assertions.assertTrue;
Expand All @@ -69,6 +71,8 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines()

OzoneConfiguration config = new OzoneConfiguration();
MockNodeManager nodeManager = new MockNodeManager(true, 0);
ContainerManager containerManager = Mockito.mock(ContainerManager.class);
Mockito.when(containerManager.getContainers()).thenReturn(containers);
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
// enable pipeline check
config.setBoolean(
Expand All @@ -94,7 +98,7 @@ public void testHealthyPipelineSafeModeRuleWithNoPipelines()
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
mockRatisProvider);
SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, pipelineManager, eventQueue,
config, containers, containerManager, pipelineManager, eventQueue,
serviceManager, scmContext);

HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
Expand All @@ -121,6 +125,8 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception {
// stale and last one is dead, and this repeats. So for a 12 node, 9
// healthy, 2 stale and one dead.
MockNodeManager nodeManager = new MockNodeManager(true, 12);
ContainerManager containerManager = Mockito.mock(ContainerManager.class);
Mockito.when(containerManager.getContainers()).thenReturn(containers);
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
// enable pipeline check
config.setBoolean(
Expand Down Expand Up @@ -172,7 +178,7 @@ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception {
MockRatisPipelineProvider.markPipelineHealthy(pipeline3);

SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, pipelineManager, eventQueue,
config, containers, containerManager, pipelineManager, eventQueue,
serviceManager, scmContext);

HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
Expand Down Expand Up @@ -215,6 +221,8 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines()
// stale and last one is dead, and this repeats. So for a 12 node, 9
// healthy, 2 stale and one dead.
MockNodeManager nodeManager = new MockNodeManager(true, 12);
ContainerManager containerManager = Mockito.mock(ContainerManager.class);
Mockito.when(containerManager.getContainers()).thenReturn(containers);
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, tempFile.getPath());
// enable pipeline check
config.setBoolean(
Expand Down Expand Up @@ -266,7 +274,7 @@ public void testHealthyPipelineSafeModeRuleWithMixedPipelines()
MockRatisPipelineProvider.markPipelineHealthy(pipeline3);

SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, pipelineManager, eventQueue,
config, containers, containerManager, pipelineManager, eventQueue,
serviceManager, scmContext);

HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport;
import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.ha.SCMHAManagerStub;
Expand All @@ -55,6 +56,7 @@
import org.apache.ozone.test.TestClock;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.mockito.Mockito;
import org.slf4j.LoggerFactory;

import static org.junit.jupiter.api.Assertions.assertFalse;
Expand Down Expand Up @@ -86,7 +88,8 @@ private void setup(int nodes, int pipelineFactorThreeCount,
List<ContainerInfo> containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(1));
mockNodeManager = new MockNodeManager(true, nodes);

ContainerManager containerManager = Mockito.mock(ContainerManager.class);
Mockito.when(containerManager.getContainers()).thenReturn(containers);
eventQueue = new EventQueue();
serviceManager = new SCMServiceManager();
scmContext = SCMContext.emptyContext();
Expand Down Expand Up @@ -116,7 +119,7 @@ private void setup(int nodes, int pipelineFactorThreeCount,
HddsProtos.ReplicationFactor.ONE);

SCMSafeModeManager scmSafeModeManager =
new SCMSafeModeManager(ozoneConfiguration, containers, null,
new SCMSafeModeManager(ozoneConfiguration, containers, containerManager,
pipelineManager, eventQueue, serviceManager, scmContext);

rule = scmSafeModeManager.getOneReplicaPipelineSafeModeRule();
Expand Down
Loading
Loading