|
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdds.scm.server;
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
+import java.util.UUID;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
import java.util.concurrent.atomic.AtomicLong;
|
|
@@ -60,14 +62,16 @@ public class SCMChillModeManager implements
|
|
|
private Map<String, ChillModeExitRule> exitRules = new HashMap(1);
|
|
|
private Configuration config;
|
|
|
private static final String CONT_EXIT_RULE = "ContainerChillModeRule";
|
|
|
+ private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
|
|
|
private final EventQueue eventPublisher;
|
|
|
|
|
|
SCMChillModeManager(Configuration conf, List<ContainerInfo> allContainers,
|
|
|
EventQueue eventQueue) {
|
|
|
this.config = conf;
|
|
|
this.eventPublisher = eventQueue;
|
|
|
- exitRules
|
|
|
- .put(CONT_EXIT_RULE, new ContainerChillModeRule(config, allContainers));
|
|
|
+ exitRules.put(CONT_EXIT_RULE,
|
|
|
+ new ContainerChillModeRule(config, allContainers));
|
|
|
+ exitRules.put(DN_EXIT_RULE, new DataNodeChillModeRule(config));
|
|
|
if (!conf.getBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
|
|
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT)) {
|
|
|
exitChillMode(eventQueue);
|
|
@@ -120,6 +124,7 @@ public class SCMChillModeManager implements
|
|
|
EventPublisher publisher) {
|
|
|
if (getInChillMode()) {
|
|
|
exitRules.get(CONT_EXIT_RULE).process(nodeRegistrationContainerReport);
|
|
|
+ exitRules.get(DN_EXIT_RULE).process(nodeRegistrationContainerReport);
|
|
|
validateChillModeExitRules(publisher);
|
|
|
}
|
|
|
}
|
|
@@ -187,6 +192,9 @@ public class SCMChillModeManager implements
|
|
|
|
|
|
@VisibleForTesting
|
|
|
public double getCurrentContainerThreshold() {
|
|
|
+ if (maxContainer == 0) {
|
|
|
+ return 1;
|
|
|
+ }
|
|
|
return (containerWithMinReplicas.doubleValue() / maxContainer);
|
|
|
}
|
|
|
|
|
@@ -217,6 +225,57 @@ public class SCMChillModeManager implements
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Class defining Chill mode exit criteria according to number of DataNodes
|
|
|
+ * registered with SCM.
|
|
|
+ */
|
|
|
+ public class DataNodeChillModeRule implements
|
|
|
+ ChillModeExitRule<NodeRegistrationContainerReport> {
|
|
|
+
|
|
|
+ // Min DataNodes required to exit chill mode.
|
|
|
+ private int requiredDns;
|
|
|
+ private int registeredDns = 0;
|
|
|
+ // Set to track registered DataNodes.
|
|
|
+ private HashSet<UUID> registeredDnSet;
|
|
|
+
|
|
|
+ public DataNodeChillModeRule(Configuration conf) {
|
|
|
+ requiredDns = conf
|
|
|
+ .getInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE,
|
|
|
+ HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT);
|
|
|
+ registeredDnSet = new HashSet<>(requiredDns * 2);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public boolean validate() {
|
|
|
+ return registeredDns >= requiredDns;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public double getRegisteredDataNodes() {
|
|
|
+ return registeredDns;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void process(NodeRegistrationContainerReport reportsProto) {
|
|
|
+ if (requiredDns == 0) {
|
|
|
+ // No dn check required.
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ if(inChillMode.get()) {
|
|
|
+ registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
|
|
|
+ registeredDns = registeredDnSet.size();
|
|
|
+ LOG.info("SCM in chill mode. {} DataNodes registered, {} required.",
|
|
|
+ registeredDns, requiredDns);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void cleanup() {
|
|
|
+ registeredDnSet.clear();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public static Logger getLogger() {
|
|
|
return LOG;
|