Skip to content

Commit

Permalink
HDFS-12467. Ozone: SCM: NodeManager should log when it comes out of c…
Browse files Browse the repository at this point in the history
…hill mode. Contributed by Nandakumar.
  • Loading branch information
anuengineer authored and omalley committed Apr 26, 2018
1 parent d7a94a2 commit ceec14b
Show file tree
Hide file tree
Showing 2 changed files with 47 additions and 31 deletions.
Expand Up @@ -18,7 +18,6 @@
package org.apache.hadoop.ozone.scm.node;

import com.google.common.annotations.VisibleForTesting;
import java.util.Optional;
import com.google.common.base.Preconditions;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.commons.collections.map.HashedMap;
Expand Down Expand Up @@ -62,6 +61,7 @@
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;

Expand Down Expand Up @@ -131,7 +131,17 @@ public class SCMNodeManager
private final int maxHBToProcessPerLoop;
private final String clusterID;
private final VersionInfo version;
private Optional<Boolean> inManualChillMode;
/**
* During start up of SCM, it will enter into chill mode and will be there
* until number of Datanodes registered reaches {@code chillModeNodeCount}.
* This flag is for tracking startup chill mode.
*/
private AtomicBoolean inStartupChillMode;
/**
* Administrator can put SCM into chill mode manually.
* This flag is for tracking manual chill mode.
*/
private AtomicBoolean inManualChillMode;
private final CommandQueue commandQueue;
// Node manager MXBean
private ObjectName nmInfoBean;
Expand Down Expand Up @@ -173,7 +183,10 @@ public SCMNodeManager(OzoneConfiguration conf, String clusterID)
executorService = HadoopExecutors.newScheduledThreadPool(1,
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("SCM Heartbeat Processing Thread - %d").build());
this.inManualChillMode = Optional.empty();

LOG.info("Entering startup chill mode.");
this.inStartupChillMode = new AtomicBoolean(true);
this.inManualChillMode = new AtomicBoolean(false);

Preconditions.checkState(heartbeatCheckerIntervalMs > 0);
executorService.schedule(this, heartbeatCheckerIntervalMs,
Expand Down Expand Up @@ -286,19 +299,16 @@ public void setMinimumChillModeNodes(int count) {
*/
@Override
public boolean isOutOfNodeChillMode() {
if (inManualChillMode.isPresent()) {
return !inManualChillMode.get();
}

return (totalNodes.get() >= getMinimumChillModeNodes());
return !inStartupChillMode.get() && !inManualChillMode.get();
}

/**
* Clears the manual chill mode.
*/
@Override
public void clearChillModeFlag() {
this.inManualChillMode = Optional.empty();
LOG.info("Clearing manual chill mode flag.");
this.inManualChillMode.getAndSet(false);
}

/**
Expand All @@ -307,22 +317,15 @@ public void clearChillModeFlag() {
*/
@Override
public String getChillModeStatus() {
if (inManualChillMode.isPresent() && inManualChillMode.get()) {
return "Manual chill mode is set to true." +
getNodeStatus();
}

if (inManualChillMode.isPresent() && !inManualChillMode.get()) {
return "Manual chill mode is set to false." +
getNodeStatus();
}

if (isOutOfNodeChillMode()) {
return "Out of chill mode." + getNodeStatus();
} else {
if (inStartupChillMode.get()) {
return "Still in chill mode, waiting on nodes to report in."
+ getNodeStatus();
}
if (inManualChillMode.get()) {
return "Out of startup chill mode, but in manual chill mode." +
getNodeStatus();
}
return "Out of chill mode." + getNodeStatus();
}

/**
Expand All @@ -344,27 +347,33 @@ private String getNodeStatus() {
*/
@Override
public boolean isInManualChillMode() {
if (this.inManualChillMode.isPresent()) {
return this.inManualChillMode.get();
}
return false;
return inManualChillMode.get();
}

/**
* Forcefully exits the chill mode even if we have not met the minimum
* criteria of exiting the chill mode.
* criteria of exiting the chill mode. This will exit from both startup
* and manual chill mode.
*/
@Override
public void forceExitChillMode() {
this.inManualChillMode = Optional.of(false);
if(inStartupChillMode.get()) {
LOG.info("Leaving startup chill mode.");
inStartupChillMode.getAndSet(false);
}
if(inManualChillMode.get()) {
LOG.info("Leaving manual chill mode.");
inManualChillMode.getAndSet(false);
}
}

/**
* Forcefully enters chill mode, even if all chill mode conditions are met.
*/
@Override
public void forceEnterChillMode() {
this.inManualChillMode = Optional.of(true);
LOG.info("Entering manual chill mode.");
inManualChillMode.getAndSet(true);
}

/**
Expand Down Expand Up @@ -728,6 +737,12 @@ public SCMCommand register(DatanodeID datanodeID) {
healthyNodeCount.incrementAndGet();
nodeStats.put(datanodeID.getDatanodeUuid(), new SCMNodeStat());

if(inStartupChillMode.get() &&
totalNodes.get() >= getMinimumChillModeNodes()) {
inStartupChillMode.getAndSet(false);
LOG.info("Leaving startup chill mode.");
}

// TODO: define node pool policy for non-default node pool.
// For now, all nodes are added to the "DefaultNodePool" upon registration
// if it has not been added to any node pool yet.
Expand Down
Expand Up @@ -907,7 +907,7 @@ public void testScmEnterAndExitChillMode() throws IOException,
assertTrue(nodeManager.isOutOfNodeChillMode());
status = nodeManager.getChillModeStatus();
Assert.assertThat(status,
CoreMatchers.containsString("Manual chill mode is set to false."));
CoreMatchers.containsString("Out of chill mode."));
assertFalse((nodeManager.isInManualChillMode()));


Expand All @@ -916,7 +916,8 @@ public void testScmEnterAndExitChillMode() throws IOException,
assertFalse(nodeManager.isOutOfNodeChillMode());
status = nodeManager.getChillModeStatus();
Assert.assertThat(status,
CoreMatchers.containsString("Manual chill mode is set to true."));
CoreMatchers.containsString("Out of startup chill mode," +
" but in manual chill mode."));
assertTrue((nodeManager.isInManualChillMode()));


Expand Down

0 comments on commit ceec14b

Please sign in to comment.