Merge branch 'master' into HDDS-1880-Decom
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
index 1a42f3a..3c1c6ec 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/protocol/DatanodeDetails.java
@@ -57,6 +57,8 @@
   private long setupTime;
   private String revision;
   private String buildDate;
+  private HddsProtos.NodeOperationalState persistedOpState;
+  private long persistedOpStateExpiryEpochSec = 0;
 
   /**
    * Constructs DatanodeDetails instance. DatanodeDetails.Builder is used
@@ -71,11 +73,16 @@
    * @param setupTime the setup time of DataNode
    * @param revision DataNodes's revision
    * @param buildDate DataNodes's build timestamp
+   * @param persistedOpState Operational State stored on DN.
+   * @param persistedOpStateExpiryEpochSec Seconds after the epoch the stored
+   *                                       state should expire.
    */
   @SuppressWarnings("parameternumber")
   private DatanodeDetails(UUID uuid, String ipAddress, String hostName,
       String networkLocation, List<Port> ports, String certSerialId,
-      String version, long setupTime, String revision, String buildDate) {
+      String version, long setupTime, String revision, String buildDate,
+      HddsProtos.NodeOperationalState persistedOpState,
+      long persistedOpStateExpiryEpochSec) {
     super(hostName, networkLocation, NetConstants.NODE_COST_DEFAULT);
     this.uuid = uuid;
     this.uuidString = uuid.toString();
@@ -87,6 +94,8 @@
     this.setupTime = setupTime;
     this.revision = revision;
     this.buildDate = buildDate;
+    this.persistedOpState = persistedOpState;
+    this.persistedOpStateExpiryEpochSec = persistedOpStateExpiryEpochSec;
   }
 
   public DatanodeDetails(DatanodeDetails datanodeDetails) {
@@ -103,6 +112,9 @@
     this.setupTime = datanodeDetails.setupTime;
     this.revision = datanodeDetails.revision;
     this.buildDate = datanodeDetails.buildDate;
+    this.persistedOpState = datanodeDetails.getPersistedOpState();
+    this.persistedOpStateExpiryEpochSec =
+        datanodeDetails.getPersistedOpStateExpiryEpochSec();
   }
 
   /**
@@ -181,6 +193,46 @@
   }
 
   /**
+   * Return the persistedOpState. If the stored value is null, return the
+   * default value of IN_SERVICE.
+   *
+   * @return The OperationalState persisted on the datanode.
+   */
+  public HddsProtos.NodeOperationalState getPersistedOpState() {
+    if (persistedOpState == null) {
+      return HddsProtos.NodeOperationalState.IN_SERVICE;
+    } else {
+      return persistedOpState;
+    }
+  }
+
+  /**
+   * Set the persistedOpState for this instance.
+   *
+   * @param state The new operational state.
+   */
+  public void setPersistedOpState(HddsProtos.NodeOperationalState state) {
+    this.persistedOpState = state;
+  }
+
+  /**
+   * Get the persistedOpStateExpiryEpochSec for the instance.
+   * @return Seconds from the epoch when the operational state should expire.
+   */
+  public long getPersistedOpStateExpiryEpochSec() {
+    return persistedOpStateExpiryEpochSec;
+  }
+
+  /**
+   * Set persistedOpStateExpiryEpochSec.
+   * @param expiry The number of second after the epoch the operational state
+   *               should expire.
+   */
+  public void setPersistedOpStateExpiryEpochSec(long expiry) {
+    this.persistedOpStateExpiryEpochSec = expiry;
+  }
+
+  /**
    * Given the name returns port number, null if the asked port is not found.
    *
    * @param name Name of the port
@@ -231,6 +283,13 @@
     if (datanodeDetailsProto.hasNetworkLocation()) {
       builder.setNetworkLocation(datanodeDetailsProto.getNetworkLocation());
     }
+    if (datanodeDetailsProto.hasPersistedOpState()) {
+      builder.setPersistedOpState(datanodeDetailsProto.getPersistedOpState());
+    }
+    if (datanodeDetailsProto.hasPersistedOpStateExpiry()) {
+      builder.setPersistedOpStateExpiry(
+          datanodeDetailsProto.getPersistedOpStateExpiry());
+    }
     return builder.build();
   }
 
@@ -294,6 +353,10 @@
     if (!Strings.isNullOrEmpty(getNetworkLocation())) {
       builder.setNetworkLocation(getNetworkLocation());
     }
+    if (persistedOpState != null) {
+      builder.setPersistedOpState(persistedOpState);
+    }
+    builder.setPersistedOpStateExpiry(persistedOpStateExpiryEpochSec);
 
     for (Port port : ports) {
       builder.addPorts(HddsProtos.Port.newBuilder()
@@ -342,6 +405,8 @@
         ", networkLocation: " +
         getNetworkLocation() +
         ", certSerialId: " + certSerialId +
+        ", persistedOpState: " + persistedOpState +
+        ", persistedOpStateExpiryEpochSec: " + persistedOpStateExpiryEpochSec +
         "}";
   }
 
@@ -385,6 +450,8 @@
     private long setupTime;
     private String revision;
     private String buildDate;
+    private HddsProtos.NodeOperationalState persistedOpState;
+    private long persistedOpStateExpiryEpochSec = 0;
 
     /**
      * Default private constructor. To create Builder instance use
@@ -412,6 +479,9 @@
       this.setupTime = details.getSetupTime();
       this.revision = details.getRevision();
       this.buildDate = details.getBuildDate();
+      this.persistedOpState = details.getPersistedOpState();
+      this.persistedOpStateExpiryEpochSec =
+          details.getPersistedOpStateExpiryEpochSec();
       return this;
     }
 
@@ -542,6 +612,31 @@
       return this;
     }
 
+    /*
+     * Adds persistedOpState.
+     *
+     * @param state The operational state persisted on the datanode
+     *
+     * @return DatanodeDetails.Builder
+     */
+    public Builder setPersistedOpState(HddsProtos.NodeOperationalState state){
+      this.persistedOpState = state;
+      return this;
+    }
+
+    /*
+     * Adds persistedOpStateExpiryEpochSec.
+     *
+     * @param expiry The seconds after the epoch the operational state should
+     *              expire.
+     *
+     * @return DatanodeDetails.Builder
+     */
+    public Builder setPersistedOpStateExpiry(long expiry){
+      this.persistedOpStateExpiryEpochSec = expiry;
+      return this;
+    }
+
     /**
      * Builds and returns DatanodeDetails instance.
      *
@@ -553,8 +648,8 @@
         networkLocation = NetConstants.DEFAULT_RACK;
       }
       DatanodeDetails dn = new DatanodeDetails(id, ipAddress, hostName,
-          networkLocation, ports, certSerialId,
-          version, setupTime, revision, buildDate);
+          networkLocation, ports, certSerialId, version, setupTime, revision,
+          buildDate, persistedOpState, persistedOpStateExpiryEpochSec);
       if (networkName != null) {
         dn.setNetworkName(networkName);
       }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
index 7b01e07..60f5c5f 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfigKeys.java
@@ -364,6 +364,11 @@
   public static final String HDDS_TRACING_ENABLED = "hdds.tracing.enabled";
   public static final boolean HDDS_TRACING_ENABLED_DEFAULT = false;
 
+  public static final String OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL =
+      "ozone.scm.datanode.admin.monitor.interval";
+  public static final String OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL_DEFAULT =
+      "30s";
+
   /**
    * Never constructed.
    */
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index e4369fa..5a77bdb 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -144,15 +144,52 @@
       String owner) throws IOException;
 
   /**
-   * Returns a set of Nodes that meet a query criteria.
-   * @param nodeStatuses - Criteria that we want the node to have.
+   * Returns a set of Nodes that meet a query criteria. Passing null for opState
+   * or nodeState acts like a wild card, returning all nodes in that state.
+   * @param opState - Operational State of the node, eg IN_SERVICE,
+   *                DECOMMISSIONED, etc
+   * @param nodeState - Health of the nodeCriteria that we want the node to
+   *                  have, eg HEALTHY, STALE etc
    * @param queryScope - Query scope - Cluster or pool.
    * @param poolName - if it is pool, a pool name is required.
    * @return A set of nodes that meet the requested criteria.
    * @throws IOException
    */
-  List<HddsProtos.Node> queryNode(HddsProtos.NodeState nodeStatuses,
-      HddsProtos.QueryScope queryScope, String poolName) throws IOException;
+  List<HddsProtos.Node> queryNode(HddsProtos.NodeOperationalState opState,
+      HddsProtos.NodeState nodeState, HddsProtos.QueryScope queryScope,
+      String poolName) throws IOException;
+
+  /**
+   * Allows a list of hosts to be decommissioned. The hosts are identified
+   * by their hostname and optionally port in the format foo.com:port.
+   * @param hosts A list of hostnames, optionally with port
+   * @throws IOException
+   */
+  void decommissionNodes(List<String> hosts) throws IOException;
+
+  /**
+   * Allows a list of hosts in maintenance or decommission states to be placed
+   * back in service. The hosts are identified by their hostname and optionally
+   * port in the format foo.com:port.
+   * @param hosts A list of hostnames, optionally with port
+   * @throws IOException
+   */
+  void recommissionNodes(List<String> hosts) throws IOException;
+
+  /**
+   * Place the list of datanodes into maintenance mode. If a non-zero endDtm
+   * is passed, the hosts will automatically exit maintenance mode after the
+   * given time has passed. Passing an end time of zero means the hosts will
+   * remain in maintenance indefinitely.
+   * The hosts are identified by their hostname and optionally port in the
+   * format foo.com:port.
+   * @param hosts A list of hostnames, optionally with port
+   * @param endHours The number of hours from now which maintenance will end or
+   *                 zero if maintenance must be manually ended.
+   * @throws IOException
+   */
+  void startMaintenanceNodes(List<String> hosts, int endHours)
+      throws IOException;
 
   /**
    * Creates a specified replication pipeline.
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index 1d58c97..1b63f50 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -119,12 +119,22 @@
   void deleteContainer(long containerID) throws IOException;
 
   /**
-   *  Queries a list of Node Statuses.
-   * @param state
+   *  Queries a list of Node Statuses. Passing a null for either opState or
+   *  state acts like a wildcard returning all nodes in that state.
+   * @param opState The node operational state
+   * @param state The node health
    * @return List of Datanodes.
    */
-  List<HddsProtos.Node> queryNode(HddsProtos.NodeState state,
-      HddsProtos.QueryScope queryScope, String poolName) throws IOException;
+  List<HddsProtos.Node> queryNode(HddsProtos.NodeOperationalState opState,
+      HddsProtos.NodeState state, HddsProtos.QueryScope queryScope,
+      String poolName) throws IOException;
+
+  void decommissionNodes(List<String> nodes) throws IOException;
+
+  void recommissionNodes(List<String> nodes) throws IOException;
+
+  void startMaintenanceNodes(List<String> nodes, int endInHours)
+      throws IOException;
 
   /**
    * Close a container.
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index 4853978..fb5ea58 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -2407,6 +2407,17 @@
     </description>
   </property>
   <property>
+    <name>ozone.scm.datanode.admin.monitor.interval</name>
+    <value>30s</value>
+    <tag>SCM</tag>
+    <description>
+      This sets how frequently the datanode admin monitor runs to check for
+      nodes added to the admin workflow or removed from it. The progress
+      of decommissioning and entering maintenance nodes is also checked to see
+      if they have completed.
+    </description>
+  </property>
+  <property>
     <name>ozone.client.list.trash.keys.max</name>
     <value>1000</value>
     <tag>OZONE, CLIENT</tag>
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/MockDatanodeDetails.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/MockDatanodeDetails.java
index 06a1bf0..41ae6ec 100644
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/MockDatanodeDetails.java
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/protocol/MockDatanodeDetails.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdds.protocol;
 
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
 import java.io.IOException;
 import java.net.ServerSocket;
 import java.util.Random;
@@ -101,6 +103,8 @@
         .addPort(ratisPort)
         .addPort(restPort)
         .setNetworkLocation(networkLocation)
+        .setPersistedOpState(HddsProtos.NodeOperationalState.IN_SERVICE)
+        .setPersistedOpStateExpiry(0)
         .build();
   }
 
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java
index 44a12c2..2d4ece2 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/DatanodeIdYaml.java
@@ -30,6 +30,7 @@
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.commons.collections.MapUtils;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.yaml.snakeyaml.DumperOptions;
 import org.yaml.snakeyaml.Yaml;
 
@@ -83,6 +84,12 @@
           .setIpAddress(datanodeDetailsYaml.getIpAddress())
           .setHostName(datanodeDetailsYaml.getHostName())
           .setCertSerialId(datanodeDetailsYaml.getCertSerialId());
+      if (datanodeDetailsYaml.getPersistedOpState() != null) {
+        builder.setPersistedOpState(HddsProtos.NodeOperationalState.valueOf(
+            datanodeDetailsYaml.getPersistedOpState()));
+      }
+      builder.setPersistedOpStateExpiry(
+          datanodeDetailsYaml.getPersistedOpStateExpiryEpochSec());
 
       if (!MapUtils.isEmpty(datanodeDetailsYaml.getPortDetails())) {
         for (Map.Entry<String, Integer> portEntry :
@@ -106,6 +113,8 @@
     private String ipAddress;
     private String hostName;
     private String certSerialId;
+    private String persistedOpState;
+    private long persistedOpStateExpiryEpochSec = 0;
     private Map<String, Integer> portDetails;
 
     public DatanodeDetailsYaml() {
@@ -114,11 +123,15 @@
 
     private DatanodeDetailsYaml(String uuid, String ipAddress,
                                 String hostName, String certSerialId,
+                                String persistedOpState,
+                                long persistedOpStateExpiryEpochSec,
                                 Map<String, Integer> portDetails) {
       this.uuid = uuid;
       this.ipAddress = ipAddress;
       this.hostName = hostName;
       this.certSerialId = certSerialId;
+      this.persistedOpState = persistedOpState;
+      this.persistedOpStateExpiryEpochSec = persistedOpStateExpiryEpochSec;
       this.portDetails = portDetails;
     }
 
@@ -138,6 +151,14 @@
       return certSerialId;
     }
 
+    public String getPersistedOpState() {
+      return persistedOpState;
+    }
+
+    public long getPersistedOpStateExpiryEpochSec() {
+      return persistedOpStateExpiryEpochSec;
+    }
+
     public Map<String, Integer> getPortDetails() {
       return portDetails;
     }
@@ -158,6 +179,14 @@
       this.certSerialId = certSerialId;
     }
 
+    public void setPersistedOpState(String persistedOpState) {
+      this.persistedOpState = persistedOpState;
+    }
+
+    public void setPersistedOpStateExpiryEpochSec(long opStateExpiryEpochSec) {
+      this.persistedOpStateExpiryEpochSec = opStateExpiryEpochSec;
+    }
+
     public void setPortDetails(Map<String, Integer> portDetails) {
       this.portDetails = portDetails;
     }
@@ -173,11 +202,17 @@
       }
     }
 
+    String persistedOpString = null;
+    if (datanodeDetails.getPersistedOpState() != null) {
+      persistedOpString = datanodeDetails.getPersistedOpState().name();
+    }
     return new DatanodeDetailsYaml(
         datanodeDetails.getUuid().toString(),
         datanodeDetails.getIpAddress(),
         datanodeDetails.getHostName(),
         datanodeDetails.getCertSerialId(),
+        persistedOpString,
+        datanodeDetails.getPersistedOpStateExpiryEpochSec(),
         portDetails);
   }
 }
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
index f07cc3a..f2f28a1 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/DatanodeStateMachine.java
@@ -43,6 +43,7 @@
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.DeleteBlocksCommandHandler;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.DeleteContainerCommandHandler;
 import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReplicateContainerCommandHandler;
+import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.SetNodeOperationalStateCommandHandler;
 import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
 import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
 import org.apache.hadoop.ozone.container.replication.ContainerReplicator;
@@ -145,6 +146,7 @@
             dnConf.getContainerDeleteThreads()))
         .addHandler(new ClosePipelineCommandHandler())
         .addHandler(new CreatePipelineCommandHandler(conf))
+        .addHandler(new SetNodeOperationalStateCommandHandler(conf))
         .setConnectionManager(connectionManager)
         .setContainer(container)
         .setContext(context)
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java
new file mode 100644
index 0000000..4a46d5f
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/SetNodeOperationalStateCommandHandler.java
@@ -0,0 +1,157 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.container.common.statemachine.commandhandler;
+
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
+import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.protocol.proto.
+    StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+
+/**
+ * Handle the SetNodeOperationalStateCommand sent from SCM to the datanode
+ * to persist the current operational state.
+ */
+public class SetNodeOperationalStateCommandHandler implements CommandHandler {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SetNodeOperationalStateCommandHandler.class);
+  private final ConfigurationSource conf;
+  private final AtomicInteger invocationCount = new AtomicInteger(0);
+  private final AtomicLong totalTime = new AtomicLong(0);
+
+  /**
+   * Set Node State command handler.
+   *
+   * @param conf - Configuration for the datanode.
+   */
+  public SetNodeOperationalStateCommandHandler(ConfigurationSource conf) {
+    this.conf = conf;
+  }
+
+  /**
+   * Handles a given SCM command.
+   *
+   * @param command - SCM Command
+   * @param container - Ozone Container.
+   * @param context - Current Context.
+   * @param connectionManager - The SCMs that we are talking to.
+   */
+  @Override
+  public void handle(SCMCommand command, OzoneContainer container,
+      StateContext context, SCMConnectionManager connectionManager) {
+    long startTime = Time.monotonicNow();
+    invocationCount.incrementAndGet();
+    StorageContainerDatanodeProtocolProtos.SetNodeOperationalStateCommandProto
+        setNodeCmdProto = null;
+
+    if (command.getType() != Type.setNodeOperationalStateCommand) {
+      LOG.warn("Skipping handling command, expected command "
+              + "type {} but found {}",
+          Type.setNodeOperationalStateCommand, command.getType());
+      return;
+    }
+    SetNodeOperationalStateCommand setNodeCmd =
+        (SetNodeOperationalStateCommand) command;
+    setNodeCmdProto = setNodeCmd.getProto();
+    DatanodeDetails dni = context.getParent().getDatanodeDetails();
+    dni.setPersistedOpState(setNodeCmdProto.getNodeOperationalState());
+    dni.setPersistedOpStateExpiryEpochSec(
+        setNodeCmd.getStateExpiryEpochSeconds());
+    try {
+      persistDatanodeDetails(dni);
+    } catch (IOException ioe) {
+      LOG.error("Failed to persist the datanode state", ioe);
+      // TODO - this should probably be raised, but it will break the command
+      //      handler interface.
+    }
+    totalTime.addAndGet(Time.monotonicNow() - startTime);
+  }
+
+  // TODO - this duplicates code in HddsDatanodeService and InitDatanodeState
+  //        Need to refactor.
+  private void persistDatanodeDetails(DatanodeDetails dnDetails)
+      throws IOException {
+    String idFilePath = HddsServerUtil.getDatanodeIdFilePath(conf);
+    if (idFilePath == null || idFilePath.isEmpty()) {
+      LOG.error("A valid path is needed for config setting {}",
+          ScmConfigKeys.OZONE_SCM_DATANODE_ID_DIR);
+      throw new IllegalArgumentException(
+          ScmConfigKeys.OZONE_SCM_DATANODE_ID_DIR +
+              " must be defined. See" +
+              " https://wiki.apache.org/hadoop/Ozone#Configuration" +
+              " for details on configuring Ozone.");
+    }
+
+    Preconditions.checkNotNull(idFilePath);
+    File idFile = new File(idFilePath);
+    ContainerUtils.writeDatanodeDetailsTo(dnDetails, idFile);
+  }
+
+  /**
+   * Returns the command type that this command handler handles.
+   *
+   * @return Type
+   */
+  @Override
+  public StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type
+      getCommandType() {
+    return Type.setNodeOperationalStateCommand;
+  }
+
+  /**
+   * Returns number of times this handler has been invoked.
+   *
+   * @return int
+   */
+  @Override
+  public int getInvocationCount() {
+    return invocationCount.intValue();
+  }
+
+  /**
+   * Returns the average time this function takes to run.
+   *
+   * @return long
+   */
+  @Override
+  public long getAverageRunTime() {
+    final int invocations = invocationCount.get();
+    return invocations == 0 ?
+        0 : totalTime.get() / invocations;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
index da2034d..cea4295 100644
--- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/states/endpoint/HeartbeatEndpointTask.java
@@ -52,6 +52,7 @@
 import org.apache.hadoop.ozone.protocol.commands.DeleteContainerCommand;
 import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
 
+import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -331,6 +332,17 @@
         }
         this.context.addCommand(closePipelineCommand);
         break;
+      case setNodeOperationalStateCommand:
+        SetNodeOperationalStateCommand setNodeOperationalStateCommand =
+            SetNodeOperationalStateCommand.getFromProtobuf(
+                commandResponseProto.getSetNodeOperationalStateCommandProto());
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Received SCM set operational state command. State: {} " +
+              "Expiry: {}", setNodeOperationalStateCommand.getOpState(),
+              setNodeOperationalStateCommand.getStateExpiryEpochSeconds());
+        }
+        this.context.addCommand(setNodeOperationalStateCommand);
+        break;
       default:
         throw new IllegalArgumentException("Unknown response : "
             + commandResponseProto.getCommandType().name());
diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SetNodeOperationalStateCommand.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SetNodeOperationalStateCommand.java
new file mode 100644
index 0000000..3ff7949
--- /dev/null
+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/protocol/commands/SetNodeOperationalStateCommand.java
@@ -0,0 +1,89 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.protocol.commands;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.SetNodeOperationalStateCommandProto;
+
+/**
+ * A command used to persist the current node operational state on the datanode.
+ */
+public class SetNodeOperationalStateCommand
+    extends SCMCommand<SetNodeOperationalStateCommandProto> {
+
+  private final HddsProtos.NodeOperationalState opState;
+  private long stateExpiryEpochSeconds;
+
+  /**
+   * Ctor that creates a SetNodeOperationalStateCommand.
+   *
+   * @param id    - Command ID. Something a time stamp would suffice.
+   * @param state - OperationalState that want the node to be set into.
+   * @param stateExpiryEpochSeconds The epoch time when the state should
+   *                                expire, or zero for the state to remain
+   *                                indefinitely.
+   */
+  public SetNodeOperationalStateCommand(long id,
+      HddsProtos.NodeOperationalState state, long stateExpiryEpochSeconds) {
+    super(id);
+    this.opState = state;
+    this.stateExpiryEpochSeconds = stateExpiryEpochSeconds;
+  }
+
+  /**
+   * Returns the type of this command.
+   *
+   * @return Type  - This is setNodeOperationalStateCommand.
+   */
+  @Override
+  public SCMCommandProto.Type getType() {
+    return SCMCommandProto.Type.setNodeOperationalStateCommand;
+  }
+
+  /**
+   * Gets the protobuf message of this object.
+   *
+   * @return A protobuf message.
+   */
+  @Override
+  public SetNodeOperationalStateCommandProto getProto() {
+    return SetNodeOperationalStateCommandProto.newBuilder()
+        .setCmdId(getId())
+        .setNodeOperationalState(opState)
+        .setStateExpiryEpochSeconds(stateExpiryEpochSeconds).build();
+  }
+
+  public HddsProtos.NodeOperationalState getOpState() {
+    return opState;
+  }
+
+  public long getStateExpiryEpochSeconds() {
+    return stateExpiryEpochSeconds;
+  }
+
+  public static SetNodeOperationalStateCommand getFromProtobuf(
+      SetNodeOperationalStateCommandProto cmdProto) {
+    Preconditions.checkNotNull(cmdProto);
+    return new SetNodeOperationalStateCommand(cmdProto.getCmdId(),
+        cmdProto.getNodeOperationalState(),
+        cmdProto.getStateExpiryEpochSeconds());
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
index 0733940..a31b5fc 100644
--- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
@@ -62,6 +62,9 @@
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ScmContainerLocationResponse;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartMaintenanceNodesRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionNodesRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.RecommissionNodesRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -285,23 +288,89 @@
   }
 
   /**
-   * Queries a list of Node Statuses.
+   * Queries a list of Nodes based on their operational state or health state.
+   * Passing a null for either value acts as a wildcard for that state.
+   *
+   * @param opState The operation state of the node
+   * @param nodeState The health of the node
+   * @return List of Datanodes.
    */
   @Override
-  public List<HddsProtos.Node> queryNode(HddsProtos.NodeState
-      nodeStatuses, HddsProtos.QueryScope queryScope, String poolName)
+  public List<HddsProtos.Node> queryNode(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState
+      nodeState, HddsProtos.QueryScope queryScope, String poolName)
       throws IOException {
     // TODO : We support only cluster wide query right now. So ignoring checking
     // queryScope and poolName
-    Preconditions.checkNotNull(nodeStatuses);
-    NodeQueryRequestProto request = NodeQueryRequestProto.newBuilder()
-        .setState(nodeStatuses)
+    NodeQueryRequestProto.Builder builder = NodeQueryRequestProto.newBuilder()
         .setTraceID(TracingUtil.exportCurrentSpan())
-        .setScope(queryScope).setPoolName(poolName).build();
+        .setScope(queryScope).setPoolName(poolName);
+    if (opState != null) {
+      builder.setOpState(opState);
+    }
+    if (nodeState != null) {
+      builder.setState(nodeState);
+    }
+    NodeQueryRequestProto request = builder.build();
     NodeQueryResponseProto response = submitRequest(Type.QueryNode,
-        builder -> builder.setNodeQueryRequest(request)).getNodeQueryResponse();
+        builder1 -> builder1.setNodeQueryRequest(request))
+        .getNodeQueryResponse();
     return response.getDatanodesList();
+  }
 
+  /**
+   * Attempts to decommission the list of nodes.
+   * @param nodes The list of hostnames or hostname:ports to decommission
+   * @throws IOException
+   */
+  @Override
+  public void decommissionNodes(List<String> nodes) throws IOException {
+    Preconditions.checkNotNull(nodes);
+    DecommissionNodesRequestProto request =
+        DecommissionNodesRequestProto.newBuilder()
+        .addAllHosts(nodes)
+        .build();
+    submitRequest(Type.DecommissionNodes,
+        builder -> builder.setDecommissionNodesRequest(request));
+  }
+
+  /**
+   * Attempts to recommission the list of nodes.
+   * @param nodes The list of hostnames or hostname:ports to recommission
+   * @throws IOException
+   */
+  @Override
+  public void recommissionNodes(List<String> nodes) throws IOException {
+    Preconditions.checkNotNull(nodes);
+    RecommissionNodesRequestProto request =
+        RecommissionNodesRequestProto.newBuilder()
+            .addAllHosts(nodes)
+            .build();
+    submitRequest(Type.RecommissionNodes,
+        builder -> builder.setRecommissionNodesRequest(request));
+  }
+
+  /**
+   * Attempts to put the list of nodes into maintenance mode.
+   *
+   * @param nodes The list of hostnames or hostname:ports to put into
+   *              maintenance
+   * @param endInHours A number of hours from now where the nodes will be taken
+   *                   out of maintenance automatically. Passing zero will
+   *                   allow the nodes to stay in maintenance indefinitely
+   * @throws IOException
+   */
+  @Override
+  public void startMaintenanceNodes(List<String> nodes, int endInHours)
+      throws IOException {
+    Preconditions.checkNotNull(nodes);
+    StartMaintenanceNodesRequestProto request =
+        StartMaintenanceNodesRequestProto.newBuilder()
+            .addAllHosts(nodes)
+            .setEndInHours(endInHours)
+            .build();
+    submitRequest(Type.StartMaintenanceNodes,
+        builder -> builder.setStartMaintenanceNodesRequest(request));
   }
 
   /**
diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
index 91dbebe..e9e0c73 100644
--- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
+++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto
@@ -62,6 +62,9 @@
   optional GetPipelineRequestProto getPipelineRequest = 24;
   optional GetContainerWithPipelineBatchRequestProto getContainerWithPipelineBatchRequest = 25;
   optional GetSafeModeRuleStatusesRequestProto getSafeModeRuleStatusesRequest = 26;
+  optional DecommissionNodesRequestProto decommissionNodesRequest = 27;
+  optional RecommissionNodesRequestProto recommissionNodesRequest = 28;
+  optional StartMaintenanceNodesRequestProto startMaintenanceNodesRequest = 29;
 }
 
 message ScmContainerLocationResponse {
@@ -96,6 +99,9 @@
   optional GetPipelineResponseProto getPipelineResponse = 24;
   optional GetContainerWithPipelineBatchResponseProto getContainerWithPipelineBatchResponse = 25;
   optional GetSafeModeRuleStatusesResponseProto getSafeModeRuleStatusesResponse = 26;
+  optional DecommissionNodesResponseProto decommissionNodesResponse = 27;
+  optional RecommissionNodesResponseProto recommissionNodesResponse = 28;
+  optional StartMaintenanceNodesResponseProto startMaintenanceNodesResponse = 29;
   enum Status {
     OK = 1;
     CONTAINER_ALREADY_EXISTS = 2;
@@ -126,6 +132,9 @@
   GetPipeline = 19;
   GetContainerWithPipelineBatch = 20;
   GetSafeModeRuleStatuses = 21;
+  DecommissionNodes = 22;
+  RecommissionNodes = 23;
+  StartMaintenanceNodes = 24;
 }
 
 /**
@@ -229,16 +238,51 @@
  match the NodeState that we are requesting.
 */
 message NodeQueryRequestProto {
-  required NodeState state = 1;
+  optional NodeState state = 1;
   required QueryScope scope = 2;
   optional string poolName = 3; // if scope is pool, then pool name is needed.
   optional string traceID = 4;
+  optional NodeOperationalState opState = 5;
 }
 
 message NodeQueryResponseProto {
   repeated Node datanodes = 1;
 }
 
+/*
+  Decommission a list of hosts
+*/
+message DecommissionNodesRequestProto {
+  repeated string hosts = 1;
+}
+
+message DecommissionNodesResponseProto {
+  // empty response
+}
+
+/*
+  Recommission a list of hosts in maintenance or decommission states
+*/
+message RecommissionNodesRequestProto {
+  repeated string hosts = 1;
+}
+
+message RecommissionNodesResponseProto {
+  // empty response
+}
+
+/*
+  Place a list of hosts into maintenance mode
+*/
+message StartMaintenanceNodesRequestProto {
+  repeated string hosts = 1;
+  optional int64 endInHours = 2;
+}
+
+message StartMaintenanceNodesResponseProto {
+  // empty response
+}
+
 /**
   Request to create a replication pipeline.
  */
@@ -349,5 +393,4 @@
  */
 service StorageContainerLocationProtocolService {
   rpc submitRequest (ScmContainerLocationRequest) returns (ScmContainerLocationResponse);
-
 }
diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
index b43a74c..2d06b3c 100644
--- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto
+++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto
@@ -43,6 +43,8 @@
     // network name, can be Ip address or host name, depends
     optional string networkName = 6;
     optional string networkLocation = 7; // Network topology location
+    optional NodeOperationalState persistedOpState = 8; // The Operational state persisted in the datanode.id file
+    optional int64 persistedOpStateExpiry = 9; // The seconds after the epoch when the OpState should expire
     // TODO(runzhiwang): when uuid is gone, specify 1 as the index of uuid128 and mark as required
     optional UUID uuid128 = 100; // UUID with 128 bits assigned to the Datanode.
 }
@@ -129,8 +131,14 @@
     HEALTHY = 1;
     STALE = 2;
     DEAD = 3;
-    DECOMMISSIONING = 4;
-    DECOMMISSIONED = 5;
+}
+
+enum NodeOperationalState {
+    IN_SERVICE = 1;
+    DECOMMISSIONING = 2;
+    DECOMMISSIONED = 3;
+    ENTERING_MAINTENANCE = 4;
+    IN_MAINTENANCE = 5;
 }
 
 enum QueryScope {
@@ -141,6 +149,7 @@
 message Node {
     required DatanodeDetailsProto nodeID = 1;
     repeated NodeState nodeStates = 2;
+    repeated NodeOperationalState nodeOperationalStates = 3;
 }
 
 message NodePool {
diff --git a/hadoop-hdds/interface-client/src/main/resources/proto.lock b/hadoop-hdds/interface-client/src/main/resources/proto.lock
index 581ffaf..8bd3023 100644
--- a/hadoop-hdds/interface-client/src/main/resources/proto.lock
+++ b/hadoop-hdds/interface-client/src/main/resources/proto.lock
@@ -1292,14 +1292,6 @@
               {
                 "name": "DEAD",
                 "integer": 3
-              },
-              {
-                "name": "DECOMMISSIONING",
-                "integer": 4
-              },
-              {
-                "name": "DECOMMISSIONED",
-                "integer": 5
               }
             ]
           },
diff --git a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
index 4f610ff..505f9ca 100644
--- a/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
+++ b/hadoop-hdds/interface-server/src/main/proto/ScmServerDatanodeHeartbeatProtocol.proto
@@ -293,6 +293,7 @@
     replicateContainerCommand = 5;
     createPipelineCommand = 6;
     closePipelineCommand = 7;
+    setNodeOperationalStateCommand = 8;
   }
   // TODO: once we start using protoc 3.x, refactor this message using "oneof"
   required Type commandType = 1;
@@ -303,6 +304,7 @@
   optional ReplicateContainerCommandProto replicateContainerCommandProto = 6;
   optional CreatePipelineCommandProto createPipelineCommandProto = 7;
   optional ClosePipelineCommandProto closePipelineCommandProto = 8;
+  optional SetNodeOperationalStateCommandProto setNodeOperationalStateCommandProto = 9;
 }
 
 /**
@@ -391,6 +393,12 @@
   required int64 cmdId = 2;
 }
 
+message SetNodeOperationalStateCommandProto {
+  required  int64 cmdId = 1;
+  required  NodeOperationalState nodeOperationalState = 2;
+  required  int64 stateExpiryEpochSeconds = 3;
+}
+
 /**
  * Protocol used from a datanode to StorageContainerManager.
  *
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/SCMCommonPlacementPolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/SCMCommonPlacementPolicy.java
index dfacae0..91b5494 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/SCMCommonPlacementPolicy.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/SCMCommonPlacementPolicy.java
@@ -24,12 +24,12 @@
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
@@ -122,7 +122,7 @@
       List<DatanodeDetails> excludedNodes, List<DatanodeDetails> favoredNodes,
       int nodesRequired, final long sizeRequired) throws SCMException {
     List<DatanodeDetails> healthyNodes =
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY);
+        nodeManager.getNodes(NodeStatus.inServiceHealthy());
     if (excludedNodes != null) {
       healthyNodes.removeAll(excludedNodes);
     }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
index 6a1f740..859a732 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/SCMBlockDeletingService.java
@@ -24,12 +24,12 @@
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.hdds.scm.ScmConfig;
 import org.apache.hadoop.hdds.scm.container.ContainerManager;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.hdds.utils.BackgroundService;
 import org.apache.hadoop.hdds.utils.BackgroundTask;
@@ -115,11 +115,14 @@
       long startTime = Time.monotonicNow();
       // Scan SCM DB in HB interval and collect a throttled list of
       // to delete blocks.
+
       if (LOG.isDebugEnabled()) {
         LOG.debug("Running DeletedBlockTransactionScanner");
       }
-
-      List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
+      // TODO - DECOMM - should we be deleting blocks from decom nodes
+      //        and what about entering maintenance.
+      List<DatanodeDetails> datanodes =
+          nodeManager.getNodes(NodeStatus.inServiceHealthy());
       if (datanodes != null) {
         try {
           DatanodeDeletedBlockTransactions transactions =
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaCount.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaCount.java
new file mode 100644
index 0000000..bf8c3b9
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ContainerReplicaCount.java
@@ -0,0 +1,271 @@
+/**
+ * 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.container;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import java.util.Set;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+
+/**
+ * Immutable object that is created with a set of ContainerReplica objects and
+ * the number of in flight replica add and deletes, the container replication
+ * factor and the min count which must be available for maintenance. This
+ * information can be used to determine if the container is over or under
+ * replicated and also how many additional replicas need created or removed.
+ */
+public class ContainerReplicaCount {
+
+  private int healthyCount = 0;
+  private int decommissionCount = 0;
+  private int maintenanceCount = 0;
+  private int inFlightAdd = 0;
+  private int inFlightDel = 0;
+  private int repFactor;
+  private int minHealthyForMaintenance;
+  private ContainerInfo container;
+  private Set<ContainerReplica> replica;
+
+  public ContainerReplicaCount(ContainerInfo container,
+                               Set<ContainerReplica> replica, int inFlightAdd,
+                               int inFlightDelete, int replicationFactor,
+                               int minHealthyForMaintenance) {
+    this.healthyCount = 0;
+    this.decommissionCount = 0;
+    this.maintenanceCount = 0;
+    this.inFlightAdd = inFlightAdd;
+    this.inFlightDel = inFlightDelete;
+    this.repFactor = replicationFactor;
+    this.replica = replica;
+    this.minHealthyForMaintenance
+        = Math.min(this.repFactor, minHealthyForMaintenance);
+    this.container = container;
+
+    for (ContainerReplica cr : this.replica) {
+      HddsProtos.NodeOperationalState state =
+          cr.getDatanodeDetails().getPersistedOpState();
+      if (state == DECOMMISSIONED || state == DECOMMISSIONING) {
+        decommissionCount++;
+      } else if (state == IN_MAINTENANCE || state == ENTERING_MAINTENANCE) {
+        maintenanceCount++;
+      } else {
+        healthyCount++;
+      }
+    }
+  }
+
+  public int getHealthyCount() {
+    return healthyCount;
+  }
+
+  public int getDecommissionCount() {
+    return decommissionCount;
+  }
+
+  public int getMaintenanceCount() {
+    return maintenanceCount;
+  }
+
+  public int getReplicationFactor() {
+    return repFactor;
+  }
+
+  public ContainerInfo getContainer() {
+    return container;
+  }
+
+  public Set<ContainerReplica> getReplica() {
+    return replica;
+  }
+
+  @Override
+  public String toString() {
+    return "Container State: " +container.getState()+
+        " Replica Count: "+replica.size()+
+        " Healthy Count: "+healthyCount+
+        " Decommission Count: "+decommissionCount+
+        " Maintenance Count: "+maintenanceCount+
+        " inFlightAdd Count: "+inFlightAdd+
+        " inFightDel Count: "+inFlightDel+
+        " ReplicationFactor: "+repFactor+
+        " minMaintenance Count: "+minHealthyForMaintenance;
+  }
+
+  /**
+   * Calculates the the delta of replicas which need to be created or removed
+   * to ensure the container is correctly replicated when considered inflight
+   * adds and deletes.
+   *
+   * When considering inflight operations, it is assumed any operation will
+   * fail. However, to consider the worst case and avoid data loss, we always
+   * assume a delete will succeed and and add will fail. In this way, we will
+   * avoid scheduling too many deletes which could result in dataloss.
+   *
+   * Decisions around over-replication are made only on healthy replicas,
+   * ignoring any in maintenance and also any inflight adds. InFlight adds are
+   * ignored, as they may not complete, so if we have:
+   *
+   *     H, H, H, IN_FLIGHT_ADD
+   *
+   * And then schedule a delete, we could end up under-replicated (add fails,
+   * delete completes). It is better to let the inflight operations complete
+   * and then deal with any further over or under replication.
+   *
+   * For maintenance replicas, assuming replication factor 3, and minHealthy
+   * 2, it is possible for all 3 hosts to be put into maintenance, leaving the
+   * following (H = healthy, M = maintenance):
+   *
+   *     H, H, M, M, M
+   *
+   * Even though we are tracking 5 replicas, this is not over replicated as we
+   * ignore the maintenance copies. Later, the replicas could look like:
+   *
+   *     H, H, H, H, M
+   *
+   * At this stage, the container is over replicated by 1, so one replica can be
+   * removed.
+   *
+   * For containers which have replication factor healthy replica, we ignore any
+   * inflight add or deletes, as they may fail. Instead, wait for them to
+   * complete and then deal with any excess or deficit.
+   *
+   * For under replicated containers we do consider inflight add and delete to
+   * avoid scheduling more adds than needed. There is additional logic around
+   * containers with maintenance replica to ensure minHealthyForMaintenance
+   * replia are maintained.
+   *
+   * @return Delta of replicas needed. Negative indicates over replication and
+   *         containers should be removed. Positive indicates over replication
+   *         and zero indicates the containers has replicationFactor healthy
+   *         replica
+   */
+  public int additionalReplicaNeeded() {
+    int delta = missingReplicas();
+
+    if (delta < 0) {
+      // Over replicated, so may need to remove a container. Do not consider
+      // inFlightAdds, as they may fail, but do consider inFlightDel which
+      // will reduce the over-replication if it completes.
+      // Note this could make the delta positive if there are too many in flight
+      // deletes, which will result in an additional being scheduled.
+      return delta + inFlightDel;
+    } else {
+      // May be under or perfectly replicated.
+      // We must consider in flight add and delete when calculating the new
+      // containers needed, but we bound the lower limit at zero to allow
+      // inflight operations to complete before handling any potential over
+      // replication
+      return Math.max(0, delta - inFlightAdd + inFlightDel);
+    }
+  }
+
+  /**
+   * Returns the count of replicas which need to be created or removed to
+   * ensure the container is perfectly replicate. Inflight operations are not
+   * considered here, but the logic to determine the missing or excess counts
+   * for maintenance is present.
+   *
+   * Decisions around over-replication are made only on healthy replicas,
+   * ignoring any in maintenance. For example, if we have:
+   *
+   *     H, H, H, M, M
+   *
+   * This will not be consider over replicated until one of the Maintenance
+   * replicas moves to Healthy.
+   *
+   * If the container is perfectly replicated, zero will be return.
+   *
+   * If it is under replicated a positive value will be returned, indicating
+   * how many replicas must be added.
+   *
+   * If it is over replicated a negative value will be returned, indicating now
+   * many replicas to remove.
+   *
+   * @return Zero if the container is perfectly replicated, a positive value
+   *         for under replicated and a negative value for over replicated.
+   */
+  private int missingReplicas() {
+    int delta = repFactor - healthyCount;
+
+    if (delta < 0) {
+      // Over replicated, so may need to remove a container.
+      return delta;
+    } else if (delta > 0) {
+      // May be under-replicated, depending on maintenance.
+      delta = Math.max(0, delta - maintenanceCount);
+      int neededHealthy =
+          Math.max(0, minHealthyForMaintenance - healthyCount);
+      delta = Math.max(neededHealthy, delta);
+      return delta;
+    } else { // delta == 0
+      // We have exactly the number of healthy replicas needed.
+      return delta;
+    }
+  }
+
+  /**
+   * Return true if the container is sufficiently replicated. Decommissioning
+   * and Decommissioned containers are ignored in this check, assuming they will
+   * eventually be removed from the cluster.
+   * This check ignores inflight additions, as those replicas have not yet been
+   * created and the create could fail for some reason.
+   * The check does consider inflight deletes as there may be 3 healthy replicas
+   * now, but once the delete completes it will reduce to 2.
+   * We also assume a replica in Maintenance state cannot be removed, so the
+   * pending delete would affect only the healthy replica count.
+   *
+   * @return True if the container is sufficiently replicated and False
+   *         otherwise.
+   */
+  public boolean isSufficientlyReplicated() {
+    return missingReplicas() + inFlightDel <= 0;
+  }
+
+  /**
+   * Return true is the container is over replicated. Decommission and
+   * maintenance containers are ignored for this check.
+   * The check ignores inflight additions, as they may fail, but it does
+   * consider inflight deletes, as they would reduce the over replication when
+   * they complete.
+   *
+   * @return True if the container is over replicated, false otherwise.
+   */
+  public boolean isOverReplicated() {
+    return missingReplicas() + inFlightDel < 0;
+  }
+
+  /**
+   * Returns true if the container is healthy, meaning all replica which are not
+   * in a decommission or maintenance state are in the same state as the
+   * container and in QUASI_CLOSED or in CLOSED state.
+   *
+   * @return true if the container is healthy, false otherwise
+   */
+  public boolean isHealthy() {
+    return (container.getState() == HddsProtos.LifeCycleState.CLOSED
+        || container.getState() == HddsProtos.LifeCycleState.QUASI_CLOSED)
+        && replica.stream()
+        .filter(r -> r.getDatanodeDetails().getPersistedOpState() == IN_SERVICE)
+        .allMatch(r -> ReplicationManager.compareState(
+            container.getState(), r.getState()));
+  }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
index bab8851..ab515b8 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/ReplicationManager.java
@@ -47,6 +47,8 @@
 import org.apache.hadoop.hdds.scm.PlacementPolicy;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -107,6 +109,11 @@
   private final LockManager<ContainerID> lockManager;
 
   /**
+   * Used to lookup the health of a nodes or the nodes operational state.
+   */
+  private final NodeManager nodeManager;
+
+  /**
    * This is used for tracking container replication commands which are issued
    * by ReplicationManager and not yet complete.
    */
@@ -136,9 +143,9 @@
   private volatile boolean running;
 
   /**
-   * Used for check datanode state.
+   * Minimum number of replica in a healthy state for maintenance.
    */
-  private final NodeManager nodeManager;
+  private int minHealthyForMaintenance;
 
   /**
    * Constructs ReplicationManager instance with the given configuration.
@@ -158,11 +165,12 @@
     this.containerPlacement = containerPlacement;
     this.eventPublisher = eventPublisher;
     this.lockManager = lockManager;
+    this.nodeManager = nodeManager;
     this.conf = conf;
     this.running = false;
     this.inflightReplication = new ConcurrentHashMap<>();
     this.inflightDeletion = new ConcurrentHashMap<>();
-    this.nodeManager = nodeManager;
+    this.minHealthyForMaintenance = conf.getMaintenanceReplicaMinimum();
   }
 
   /**
@@ -258,7 +266,7 @@
    * @param id ContainerID
    */
   private void processContainer(ContainerID id) {
-    lockManager.lock(id);
+    lockManager.writeLock(id);
     try {
       final ContainerInfo container = containerManager.getContainer(id);
       final Set<ContainerReplica> replicas = containerManager
@@ -271,7 +279,7 @@
        * the replicas are not in OPEN state, send CLOSE_CONTAINER command.
        */
       if (state == LifeCycleState.OPEN) {
-        if (!isContainerHealthy(container, replicas)) {
+        if (!isOpenContainerHealthy(container, replicas)) {
           eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
         }
         return;
@@ -323,6 +331,11 @@
         return;
       }
 
+      ContainerReplicaCount replicaSet =
+          getContainerReplicaCount(container, replicas);
+      ContainerPlacementStatus placementStatus = getPlacementStatus(
+          replicas, container.getReplicationFactor().getNumber());
+
       /*
        * We don't have to take any action if the container is healthy.
        *
@@ -330,13 +343,11 @@
        * the container is either in QUASI_CLOSED or in CLOSED state and has
        * exact number of replicas in the same state.
        */
-      if (isContainerHealthy(container, replicas)) {
+      if (isContainerEmpty(container, replicas)) {
         /*
          *  If container is empty, schedule task to delete the container.
          */
-        if (isContainerEmpty(container, replicas)) {
-          deleteContainerReplicas(container, replicas);
-        }
+        deleteContainerReplicas(container, replicas);
         return;
       }
 
@@ -344,8 +355,9 @@
        * Check if the container is under replicated and take appropriate
        * action.
        */
-      if (isContainerUnderReplicated(container, replicas)) {
-        handleUnderReplicatedContainer(container, replicas);
+      if (!replicaSet.isSufficientlyReplicated()
+          || !placementStatus.isPolicySatisfied()) {
+        handleUnderReplicatedContainer(container, replicaSet, placementStatus);
         return;
       }
 
@@ -353,24 +365,26 @@
        * Check if the container is over replicated and take appropriate
        * action.
        */
-      if (isContainerOverReplicated(container, replicas)) {
-        handleOverReplicatedContainer(container, replicas);
+      if (replicaSet.isOverReplicated()) {
+        handleOverReplicatedContainer(container, replicaSet);
         return;
       }
 
       /*
-       * The container is neither under nor over replicated and the container
-       * is not healthy. This means that the container has unhealthy/corrupted
-       * replica.
+       If we get here, the container is not over replicated or under replicated
+       but it may be "unhealthy", which means it has one or more replica which
+       are not in the same state as the container itself.
        */
-      handleUnstableContainer(container, replicas);
+      if (!replicaSet.isHealthy()) {
+        handleUnstableContainer(container, replicas);
+      }
 
     } catch (ContainerNotFoundException ex) {
       LOG.warn("Missing container {}.", id);
     } catch (Exception ex) {
       LOG.warn("Process container {} error: ", id, ex);
     } finally {
-      lockManager.unlock(id);
+      lockManager.writeUnlock(id);
     }
   }
 
@@ -389,10 +403,22 @@
     if (inflightActions.containsKey(id)) {
       final List<InflightAction> actions = inflightActions.get(id);
 
-      actions.removeIf(action ->
-          nodeManager.getNodeState(action.datanode) != NodeState.HEALTHY);
-      actions.removeIf(action -> action.time < deadline);
-      actions.removeIf(filter);
+      Iterator<InflightAction> iter = actions.iterator();
+      while(iter.hasNext()) {
+        try {
+          InflightAction a = iter.next();
+          NodeState health = nodeManager.getNodeStatus(a.datanode)
+              .getHealth();
+          if (health != NodeState.HEALTHY || a.time < deadline
+              || filter.test(a)) {
+            iter.remove();
+          }
+        } catch (NodeNotFoundException e) {
+          // Should not happen, but if it does, just remove the action as the
+          // node somehow does not exist;
+          iter.remove();
+        }
+      }
       if (actions.isEmpty()) {
         inflightActions.remove(id);
       }
@@ -400,21 +426,23 @@
   }
 
   /**
-   * Returns true if the container is healthy according to ReplicationMonitor.
-   *
-   * According to ReplicationMonitor container is considered healthy if
-   * it has exact number of replicas in the same state as the container.
-   *
-   * @param container Container to check
-   * @param replicas Set of ContainerReplicas
-   * @return true if the container is healthy, false otherwise
+   * Returns the number replica which are pending creation for the given
+   * container ID.
+   * @param id The ContainerID for which to check the pending replica
+   * @return The number of inflight additions or zero if none
    */
-  private boolean isContainerHealthy(final ContainerInfo container,
-                                     final Set<ContainerReplica> replicas) {
-    return !isContainerUnderReplicated(container, replicas) &&
-        !isContainerOverReplicated(container, replicas) &&
-        replicas.stream().allMatch(
-            r -> compareState(container.getState(), r.getState()));
+  private int getInflightAdd(final ContainerID id) {
+    return inflightReplication.getOrDefault(id, Collections.emptyList()).size();
+  }
+
+  /**
+   * Returns the number replica which are pending delete for the given
+   * container ID.
+   * @param id The ContainerID for which to check the pending replica
+   * @return The number of inflight deletes or zero if none
+   */
+  private int getInflightDel(final ContainerID id) {
+    return inflightDeletion.getOrDefault(id, Collections.emptyList()).size();
   }
 
   /**
@@ -433,51 +461,61 @@
   }
 
   /**
-   * Checks if the container is under replicated or not.
-   *
-   * @param container Container to check
-   * @param replicas Set of ContainerReplicas
-   * @return true if the container is under replicated, false otherwise
+   * Given a ContainerID, lookup the ContainerInfo and then return a
+   * ContainerReplicaCount object for the container.
+   * @param containerID The ID of the container
+   * @return ContainerReplicaCount for the given container
+   * @throws ContainerNotFoundException
    */
-  private boolean isContainerUnderReplicated(final ContainerInfo container,
-      final Set<ContainerReplica> replicas) {
-    if (container.getState() != LifeCycleState.CLOSED &&
-        container.getState() != LifeCycleState.QUASI_CLOSED) {
-      return false;
+  public ContainerReplicaCount getContainerReplicaCount(ContainerID containerID)
+      throws ContainerNotFoundException {
+    ContainerInfo container = containerManager.getContainer(containerID);
+    return getContainerReplicaCount(container);
+  }
+
+  /**
+   * Given a container, obtain the set of known replica for it, and return a
+   * ContainerReplicaCount object. This object will contain the set of replica
+   * as well as all information required to determine if the container is over
+   * or under replicated, including the delta of replica required to repair the
+   * over or under replication.
+   *
+   * @param container The container to create a ContainerReplicaCount for
+   * @return ContainerReplicaCount representing the replicated state of the
+   *         container.
+   * @throws ContainerNotFoundException
+   */
+  public ContainerReplicaCount getContainerReplicaCount(ContainerInfo container)
+      throws ContainerNotFoundException {
+    lockManager.readLock(container.containerID());
+    try {
+      final Set<ContainerReplica> replica = containerManager
+          .getContainerReplicas(container.containerID());
+      return getContainerReplicaCount(container, replica);
+    } finally {
+      lockManager.readUnlock(container.containerID());
     }
-    boolean misReplicated = !getPlacementStatus(
-        replicas, container.getReplicationFactor().getNumber())
-        .isPolicySatisfied();
-    return container.getReplicationFactor().getNumber() >
-        getReplicaCount(container.containerID(), replicas) || misReplicated;
   }
 
   /**
-   * Checks if the container is over replicated or not.
+   * Given a container and its set of replicas, create and return a
+   * ContainerReplicaCount representing the container.
    *
-   * @param container Container to check
-   * @param replicas Set of ContainerReplicas
-   * @return true if the container if over replicated, false otherwise
+   * @param container The container for which to construct a
+   *                  ContainerReplicaCount
+   * @param replica The set of existing replica for this container
+   * @return ContainerReplicaCount representing the current state of the
+   *         container
    */
-  private boolean isContainerOverReplicated(final ContainerInfo container,
-      final Set<ContainerReplica> replicas) {
-    return container.getReplicationFactor().getNumber() <
-        getReplicaCount(container.containerID(), replicas);
-  }
-
-  /**
-   * Returns the replication count of the given container. This also
-   * considers inflight replication and deletion.
-   *
-   * @param id ContainerID
-   * @param replicas Set of existing replicas
-   * @return number of estimated replicas for this container
-   */
-  private int getReplicaCount(final ContainerID id,
-                              final Set<ContainerReplica> replicas) {
-    return replicas.size()
-        + inflightReplication.getOrDefault(id, Collections.emptyList()).size()
-        - inflightDeletion.getOrDefault(id, Collections.emptyList()).size();
+  private ContainerReplicaCount getContainerReplicaCount(
+      ContainerInfo container, Set<ContainerReplica> replica) {
+    return new ContainerReplicaCount(
+        container,
+        replica,
+        getInflightAdd(container.containerID()),
+        getInflightDel(container.containerID()),
+        container.getReplicationFactor().getNumber(),
+        minHealthyForMaintenance);
   }
 
   /**
@@ -601,13 +639,25 @@
    * and send replicate container command to the identified datanode(s).
    *
    * @param container ContainerInfo
-   * @param replicas Set of ContainerReplicas
+   * @param replicaSet An instance of ContainerReplicaCount, containing the
+   *                   current replica count and inflight adds and deletes
    */
   private void handleUnderReplicatedContainer(final ContainerInfo container,
-      final Set<ContainerReplica> replicas) {
+      final ContainerReplicaCount replicaSet,
+      final ContainerPlacementStatus placementStatus) {
     LOG.debug("Handling under-replicated container: {}",
         container.getContainerID());
+    Set<ContainerReplica> replicas = replicaSet.getReplica();
     try {
+
+      if (replicaSet.isSufficientlyReplicated()
+          && placementStatus.isPolicySatisfied()) {
+        LOG.info("The container {} with replicas {} is sufficiently "+
+            "replicated and is not mis-replicated",
+            container.getContainerID(), replicaSet);
+        return;
+      }
+      int repDelta = replicaSet.additionalReplicaNeeded();
       final ContainerID id = container.containerID();
       final List<DatanodeDetails> deletionInFlight = inflightDeletion
           .getOrDefault(id, Collections.emptyList())
@@ -623,6 +673,11 @@
           .filter(r ->
               r.getState() == State.QUASI_CLOSED ||
               r.getState() == State.CLOSED)
+          // Exclude stale and dead nodes. This is particularly important for
+          // maintenance nodes, as the replicas will remain present in the
+          // container manager, even when they go dead.
+          .filter(r ->
+              getNodeStatus(r.getDatanodeDetails()).isHealthy())
           .filter(r -> !deletionInFlight.contains(r.getDatanodeDetails()))
           .sorted((r1, r2) -> r2.getSequenceId().compareTo(r1.getSequenceId()))
           .map(ContainerReplica::getDatanodeDetails)
@@ -636,13 +691,12 @@
         List<DatanodeDetails> targetReplicas = new ArrayList<>(source);
         // Then add any pending additions
         targetReplicas.addAll(replicationInFlight);
-        final ContainerPlacementStatus placementStatus =
+        final ContainerPlacementStatus inFlightplacementStatus =
             containerPlacement.validateContainerPlacement(
                 targetReplicas, replicationFactor);
-        int delta = replicationFactor - getReplicaCount(id, replicas);
-        final int misRepDelta = placementStatus.misReplicationCount();
+        final int misRepDelta = inFlightplacementStatus.misReplicationCount();
         final int replicasNeeded
-            = delta < misRepDelta ? misRepDelta : delta;
+            = repDelta < misRepDelta ? misRepDelta : repDelta;
         if (replicasNeeded <= 0) {
           LOG.debug("Container {} meets replication requirement with " +
               "inflight replicas", id);
@@ -656,10 +710,10 @@
         final List<DatanodeDetails> selectedDatanodes = containerPlacement
             .chooseDatanodes(excludeList, null, replicasNeeded,
                 container.getUsedBytes());
-        if (delta > 0) {
+        if (repDelta > 0) {
           LOG.info("Container {} is under replicated. Expected replica count" +
                   " is {}, but found {}.", id, replicationFactor,
-              replicationFactor - delta);
+              replicationFactor - repDelta);
         }
         int newMisRepDelta = misRepDelta;
         if (misRepDelta > 0) {
@@ -671,7 +725,7 @@
           newMisRepDelta = containerPlacement.validateContainerPlacement(
               targetReplicas, replicationFactor).misReplicationCount();
         }
-        if (delta > 0 || newMisRepDelta < misRepDelta) {
+        if (repDelta > 0 || newMisRepDelta < misRepDelta) {
           // Only create new replicas if we are missing a replicas or
           // the number of pending mis-replication has improved. No point in
           // creating new replicas for mis-replicated containers unless it
@@ -689,7 +743,7 @@
         LOG.warn("Cannot replicate container {}, no healthy replica found.",
             container.containerID());
       }
-    } catch (IOException ex) {
+    } catch (IOException | IllegalStateException ex) {
       LOG.warn("Exception while replicating container {}.",
           container.getContainerID(), ex);
     }
@@ -701,17 +755,16 @@
    * identified datanode(s).
    *
    * @param container ContainerInfo
-   * @param replicas Set of ContainerReplicas
+   * @param replicaSet An instance of ContainerReplicaCount, containing the
+   *                   current replica count and inflight adds and deletes
    */
   private void handleOverReplicatedContainer(final ContainerInfo container,
-      final Set<ContainerReplica> replicas) {
+      final ContainerReplicaCount replicaSet) {
 
+    final Set<ContainerReplica> replicas = replicaSet.getReplica();
     final ContainerID id = container.containerID();
     final int replicationFactor = container.getReplicationFactor().getNumber();
-    // Don't consider inflight replication while calculating excess here.
-    int excess = replicas.size() - replicationFactor -
-        inflightDeletion.getOrDefault(id, Collections.emptyList()).size();
-
+    int excess = replicaSet.additionalReplicaNeeded() * -1;
     if (excess > 0) {
 
       LOG.info("Container {} is over replicated. Expected replica count" +
@@ -729,9 +782,14 @@
             .forEach(r -> uniqueReplicas
                 .putIfAbsent(r.getOriginDatanodeId(), r));
 
-        // Retain one healthy replica per origin node Id.
         eligibleReplicas.removeAll(uniqueReplicas.values());
       }
+      // Replica which are maintenance or decommissioned are not eligible to
+      // be removed, as they do not count toward over-replication and they
+      // also many not be available
+      eligibleReplicas.removeIf(r ->
+          r.getDatanodeDetails().getPersistedOpState() !=
+              HddsProtos.NodeOperationalState.IN_SERVICE);
 
       final List<ContainerReplica> unhealthyReplicas = eligibleReplicas
           .stream()
@@ -757,18 +815,18 @@
       // make the container become mis-replicated.
       if (excess > 0) {
         eligibleReplicas.removeAll(unhealthyReplicas);
-        Set<ContainerReplica> replicaSet = new HashSet<>(eligibleReplicas);
+        Set<ContainerReplica> eligibleSet = new HashSet<>(eligibleReplicas);
         ContainerPlacementStatus ps =
-            getPlacementStatus(replicaSet, replicationFactor);
+            getPlacementStatus(eligibleSet, replicationFactor);
         for (ContainerReplica r : eligibleReplicas) {
           if (excess <= 0) {
             break;
           }
           // First remove the replica we are working on from the set, and then
           // check if the set is now mis-replicated.
-          replicaSet.remove(r);
+          eligibleSet.remove(r);
           ContainerPlacementStatus nowPS =
-              getPlacementStatus(replicaSet, replicationFactor);
+              getPlacementStatus(eligibleSet, replicationFactor);
           if ((!ps.isPolicySatisfied()
                 && nowPS.actualPlacementCount() == ps.actualPlacementCount())
               || (ps.isPolicySatisfied() && nowPS.isPolicySatisfied())) {
@@ -780,7 +838,7 @@
             continue;
           }
           // If we decided not to remove this replica, put it back into the set
-          replicaSet.add(r);
+          eligibleSet.add(r);
         }
         if (excess > 0) {
           LOG.info("The container {} is over replicated with {} excess " +
@@ -957,13 +1015,27 @@
   }
 
   /**
+   * Wrap the call to nodeManager.getNodeStatus, catching any
+   * NodeNotFoundException and instead throwing an IllegalStateException.
+   * @param dn The datanodeDetails to obtain the NodeStatus for
+   * @return NodeStatus corresponding to the given Datanode.
+   */
+  private NodeStatus getNodeStatus(DatanodeDetails dn) {
+    try {
+      return nodeManager.getNodeStatus(dn);
+    } catch (NodeNotFoundException e) {
+      throw new IllegalStateException("Unable to find NodeStatus for "+dn, e);
+    }
+  }
+
+  /**
    * Compares the container state with the replica state.
    *
    * @param containerState ContainerState
    * @param replicaState ReplicaState
    * @return true if the state matches, false otherwise
    */
-  private static boolean compareState(final LifeCycleState containerState,
+  public static boolean compareState(final LifeCycleState containerState,
                                       final State replicaState) {
     switch (containerState) {
     case OPEN:
@@ -983,6 +1055,20 @@
     }
   }
 
+  /**
+   * An open container is healthy if all its replicas are in the same state as
+   * the container.
+   * @param container The container to check
+   * @param replicas The replicas belonging to the container
+   * @return True if the container is healthy, false otherwise
+   */
+  private boolean isOpenContainerHealthy(
+      ContainerInfo container, Set<ContainerReplica> replicas) {
+    LifeCycleState state = container.getState();
+    return replicas.stream()
+        .allMatch(r -> ReplicationManager.compareState(state, r.getState()));
+  }
+
   @Override
   public void getMetrics(MetricsCollector collector, boolean all) {
     collector.addRecord(ReplicationManager.class.getSimpleName())
@@ -1047,7 +1133,6 @@
             + "sent  to datanodes. After this timeout the command will be "
             + "retried.")
     private long eventTimeout = Duration.ofMinutes(30).toMillis();
-
     public void setInterval(Duration interval) {
       this.interval = interval.toMillis();
     }
@@ -1056,6 +1141,25 @@
       this.eventTimeout = timeout.toMillis();
     }
 
+    /**
+     * The number of container replica which must be available for a node to
+     * enter maintenance.
+     */
+    @Config(key = "maintenance.replica.minimum",
+        type = ConfigType.INT,
+        defaultValue = "2",
+        tags = {SCM, OZONE},
+        description = "The minimum number of container replicas which must " +
+            " be available for a node to enter maintenance. If putting a " +
+            " node into maintenance reduces the available replicas for any " +
+            " container below this level, the node will remain in the " +
+            " entering maintenance state until a new replica is created.")
+    private int maintenanceReplicaMinimum = 2;
+
+    public void setMaintenanceReplicaMinimum(int replicaCount) {
+      this.maintenanceReplicaMinimum = replicaCount;
+    }
+
     public long getInterval() {
       return interval;
     }
@@ -1063,6 +1167,10 @@
     public long getEventTimeout() {
       return eventTimeout;
     }
+
+    public int getMaintenanceReplicaMinimum() {
+      return maintenanceReplicaMinimum;
+    }
   }
 
   /**
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
index 8b40571..6f6cc54 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
@@ -162,6 +162,12 @@
       new TypedEvent<>(DatanodeDetails.class, "Dead_Node");
 
   /**
+   * This event will be triggered whenever a datanode is moved into maintenance.
+   */
+  public static final TypedEvent<DatanodeDetails> START_ADMIN_ON_NODE =
+      new TypedEvent<>(DatanodeDetails.class, "START_ADMIN_ON_NODE");
+
+  /**
    * This event will be triggered whenever a datanode is moved from non-healthy
    * state to healthy state.
    */
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitor.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitor.java
new file mode 100644
index 0000000..3466547
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitor.java
@@ -0,0 +1,34 @@
+/**
+ * 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.node;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+
+import java.util.Set;
+
+/**
+ * Interface used by the DatanodeAdminMonitor, which can be used to
+ * decommission or recommission nodes and take them in and out of maintenance.
+ */
+public interface DatanodeAdminMonitor extends Runnable {
+
+  void startMonitoring(DatanodeDetails dn);
+  void stopMonitoring(DatanodeDetails dn);
+  Set<DatanodeDetails> getTrackedNodes();
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java
new file mode 100644
index 0000000..247a307
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeAdminMonitorImpl.java
@@ -0,0 +1,371 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplicaCount;
+import org.apache.hadoop.hdds.scm.container.ReplicationManager;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * Monitor thread which watches for nodes to be decommissioned, recommissioned
+ * or placed into maintenance. Newly added nodes are queued in pendingNodes
+ * and recommissoned nodes are queued in cancelled nodes. On each monitor
+ * 'tick', the cancelled nodes are processed and removed from the monitor.
+ * Then any pending nodes are added to the trackedNodes set, where they stay
+ * until decommission or maintenance has ended.
+ * <p>
+ * Once an node is placed into tracked nodes, it goes through a workflow where
+ * the following happens:
+ * <p>
+ * 1. First an event is fired to close any pipelines on the node, which will
+ * also close any containers.
+ * 2. Next the containers on the node are obtained and checked to see if new
+ * replicas are needed. If so, the new replicas are scheduled.
+ * 3. After scheduling replication, the node remains pending until replication
+ * has completed.
+ * 4. At this stage the node will complete decommission or enter maintenance.
+ * 5. Maintenance nodes will remain tracked by this monitor until maintenance
+ * is manually ended, or the maintenance window expires.
+ */
+public class DatanodeAdminMonitorImpl implements DatanodeAdminMonitor {
+
+  private OzoneConfiguration conf;
+  private EventPublisher eventQueue;
+  private NodeManager nodeManager;
+  private ReplicationManager replicationManager;
+  private Queue<DatanodeDetails> pendingNodes = new ArrayDeque();
+  private Queue<DatanodeDetails> cancelledNodes = new ArrayDeque();
+  private Set<DatanodeDetails> trackedNodes = new HashSet<>();
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DatanodeAdminMonitorImpl.class);
+
+  public DatanodeAdminMonitorImpl(
+      OzoneConfiguration conf,
+      EventPublisher eventQueue,
+      NodeManager nodeManager,
+      ReplicationManager replicationManager) {
+    this.conf = conf;
+    this.eventQueue = eventQueue;
+    this.nodeManager = nodeManager;
+    this.replicationManager = replicationManager;
+  }
+
+  /**
+   * Add a node to the decommission or maintenance workflow. The node will be
+   * queued and added to the workflow after a defined interval.
+   *
+   * @param dn         The datanode to move into an admin state
+   */
+  @Override
+  public synchronized void startMonitoring(DatanodeDetails dn) {
+    cancelledNodes.remove(dn);
+    pendingNodes.add(dn);
+  }
+
+  /**
+   * Remove a node from the decommission or maintenance workflow, and return it
+   * to service. The node will be queued and removed from decommission or
+   * maintenance after a defined interval.
+   *
+   * @param dn The datanode for which to stop decommission or maintenance.
+   */
+  @Override
+  public synchronized void stopMonitoring(DatanodeDetails dn) {
+    pendingNodes.remove(dn);
+    cancelledNodes.add(dn);
+  }
+
+  /**
+   * Get the set of nodes which are currently tracked in the decommissioned
+   * and maintenance workflow.
+   * @return An unmodifiable set of the tracked nodes.
+   */
+  @Override
+  public synchronized Set<DatanodeDetails> getTrackedNodes() {
+    return Collections.unmodifiableSet(trackedNodes);
+  }
+
+  /**
+   * Run an iteration of the monitor. This is the main run loop, and performs
+   * the following checks:
+   * <p>
+   * 1. Check for any cancelled nodes and process them
+   * 2. Check for any newly added nodes and add them to the workflow
+   * 3. Perform checks on the transitioning nodes and move them through the
+   * workflow until they have completed decommission or maintenance
+   */
+  @Override
+  public void run() {
+    try {
+      synchronized (this) {
+        processCancelledNodes();
+        processPendingNodes();
+      }
+      processTransitioningNodes();
+      if (trackedNodes.size() > 0 || pendingNodes.size() > 0) {
+        LOG.info("There are {} nodes tracked for decommission and " +
+                "maintenance. {} pending nodes.",
+            trackedNodes.size(), pendingNodes.size());
+      }
+    } catch (Exception e) {
+      LOG.error("Caught an error in the DatanodeAdminMonitor", e);
+      // Intentionally do not re-throw, as if we do the monitor thread
+      // will not get rescheduled.
+    }
+  }
+
+  public int getPendingCount() {
+    return pendingNodes.size();
+  }
+
+  public int getCancelledCount() {
+    return cancelledNodes.size();
+  }
+
+  public int getTrackedNodeCount() {
+    return trackedNodes.size();
+  }
+
+  private void processCancelledNodes() {
+    while (!cancelledNodes.isEmpty()) {
+      DatanodeDetails dn = cancelledNodes.poll();
+      try {
+        stopTrackingNode(dn);
+        putNodeBackInService(dn);
+        LOG.info("Recommissioned node {}", dn);
+      } catch (NodeNotFoundException e) {
+        LOG.warn("Failed processing the cancel admin request for {}", dn, e);
+      }
+    }
+  }
+
+  private void processPendingNodes() {
+    while (!pendingNodes.isEmpty()) {
+      startTrackingNode(pendingNodes.poll());
+    }
+  }
+
+  private void processTransitioningNodes() {
+    Iterator<DatanodeDetails> iterator = trackedNodes.iterator();
+    while (iterator.hasNext()) {
+      DatanodeDetails dn = iterator.next();
+      try {
+        NodeStatus status = getNodeStatus(dn);
+
+        if (!shouldContinueWorkflow(dn, status)) {
+          abortWorkflow(dn);
+          iterator.remove();
+          continue;
+        }
+
+        if (status.isMaintenance()) {
+          if (status.operationalStateExpired()) {
+            completeMaintenance(dn);
+            iterator.remove();
+            continue;
+          }
+        }
+
+        if (status.isDecommissioning() || status.isEnteringMaintenance()) {
+          if (checkPipelinesClosedOnNode(dn)
+              // Ensure the DN has received and persisted the current maint
+              // state.
+              && status.getOperationalState()
+                  == dn.getPersistedOpState()
+              && checkContainersReplicatedOnNode(dn)) {
+            // CheckContainersReplicatedOnNode may take a short time to run
+            // so after it completes, re-get the nodestatus to check the health
+            // and ensure the state is still good to continue
+            status = getNodeStatus(dn);
+            if (status.isDead()) {
+              LOG.warn("Datanode {} is dead and the admin workflow cannot " +
+                  "continue. The node will be put back to IN_SERVICE and " +
+                  "handled as a dead node", dn);
+              putNodeBackInService(dn);
+              iterator.remove();
+            } else if (status.isDecommissioning()) {
+              completeDecommission(dn);
+              iterator.remove();
+            } else if (status.isEnteringMaintenance()) {
+              putIntoMaintenance(dn);
+            }
+          }
+        }
+
+      } catch (NodeNotFoundException e) {
+        LOG.error("An unexpected error occurred processing datanode {}. " +
+            "Aborting the admin workflow", dn, e);
+        abortWorkflow(dn);
+        iterator.remove();
+      }
+    }
+  }
+
+  /**
+   * Checks if a node is in an unexpected state or has gone dead while
+   * decommissioning or entering maintenance. If the node is not in a valid
+   * state to continue the admin workflow, return false, otherwise return true.
+   *
+   * @param dn         The Datanode for which to check the current state
+   * @param nodeStatus The current NodeStatus for the datanode
+   * @return True if admin can continue, false otherwise
+   */
+  private boolean shouldContinueWorkflow(DatanodeDetails dn,
+      NodeStatus nodeStatus) {
+    if (!nodeStatus.isDecommission() && !nodeStatus.isMaintenance()) {
+      LOG.warn("Datanode {} has an operational state of {} when it should " +
+              "be undergoing decommission or maintenance. Aborting admin for " +
+              "this node.", dn, nodeStatus.getOperationalState());
+      return false;
+    }
+    if (nodeStatus.isDead() && !nodeStatus.isInMaintenance()) {
+      LOG.error("Datanode {} is dead but is not IN_MAINTENANCE. Aborting the " +
+          "admin workflow for this node", dn);
+      return false;
+    }
+    return true;
+  }
+
+  private boolean checkPipelinesClosedOnNode(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    Set<PipelineID> pipelines = nodeManager.getPipelines(dn);
+    NodeStatus status = nodeManager.getNodeStatus(dn);
+    if (pipelines == null || pipelines.size() == 0
+        || status.operationalStateExpired()) {
+      return true;
+    } else {
+      LOG.info("Waiting for pipelines to close for {}. There are {} " +
+          "pipelines", dn, pipelines.size());
+      return false;
+    }
+  }
+
+  private boolean checkContainersReplicatedOnNode(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    int sufficientlyReplicated = 0;
+    int underReplicated = 0;
+    int unhealthy = 0;
+    Set<ContainerID> containers =
+        nodeManager.getContainers(dn);
+    for (ContainerID cid : containers) {
+      try {
+        ContainerReplicaCount replicaSet =
+            replicationManager.getContainerReplicaCount(cid);
+        if (replicaSet.isSufficientlyReplicated()) {
+          sufficientlyReplicated++;
+        } else {
+          underReplicated++;
+        }
+        if (!replicaSet.isHealthy()) {
+          unhealthy++;
+        }
+      } catch (ContainerNotFoundException e) {
+        LOG.warn("ContainerID {} present in node list for {} but not found " +
+            "in containerManager", cid, dn);
+      }
+    }
+    LOG.info("{} has {} sufficientlyReplicated, {} underReplicated and {} " +
+        "unhealthy containers",
+        dn, sufficientlyReplicated, underReplicated, unhealthy);
+    return underReplicated == 0 && unhealthy == 0;
+  }
+
+  private void completeDecommission(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    setNodeOpState(dn, NodeOperationalState.DECOMMISSIONED);
+    LOG.info("Datanode {} has completed the admin workflow. The operational " +
+            "state has been set to {}", dn,
+        NodeOperationalState.DECOMMISSIONED);
+  }
+
+  private void putIntoMaintenance(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    LOG.info("Datanode {} has entered maintenance", dn);
+    setNodeOpState(dn, NodeOperationalState.IN_MAINTENANCE);
+  }
+
+  private void completeMaintenance(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    // The end state of Maintenance is to put the node back IN_SERVICE, whether
+    // it is dead or not.
+    LOG.info("Datanode {} has ended maintenance automatically", dn);
+    putNodeBackInService(dn);
+  }
+
+  private void startTrackingNode(DatanodeDetails dn) {
+    eventQueue.fireEvent(SCMEvents.START_ADMIN_ON_NODE, dn);
+    trackedNodes.add(dn);
+  }
+
+  private void stopTrackingNode(DatanodeDetails dn) {
+    trackedNodes.remove(dn);
+  }
+
+  /**
+   * If we encounter an unexpected condition in maintenance, we must abort the
+   * workflow by setting the node operationalState back to IN_SERVICE and then
+   * remove the node from tracking.
+   *
+   * @param dn The datanode for which to abort tracking
+   */
+  private void abortWorkflow(DatanodeDetails dn) {
+    try {
+      putNodeBackInService(dn);
+    } catch (NodeNotFoundException e) {
+      LOG.error("Unable to set the node OperationalState for {} while " +
+          "aborting the datanode admin workflow", dn);
+    }
+  }
+
+  private void putNodeBackInService(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    setNodeOpState(dn, NodeOperationalState.IN_SERVICE);
+  }
+
+  private void setNodeOpState(DatanodeDetails dn,
+      HddsProtos.NodeOperationalState state) throws NodeNotFoundException {
+    long expiry = 0;
+    if ((state == NodeOperationalState.IN_MAINTENANCE)
+        || (state == NodeOperationalState.ENTERING_MAINTENANCE)) {
+      NodeStatus status = nodeManager.getNodeStatus(dn);
+      expiry = status.getOpStateExpiryEpochSeconds();
+    }
+    nodeManager.setNodeOperationalState(dn, state, expiry);
+  }
+
+  private NodeStatus getNodeStatus(DatanodeDetails dnd)
+      throws NodeNotFoundException {
+    return nodeManager.getNodeStatus(dnd);
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java
index 2e7bdeb..edd616f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DatanodeInfo.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdds.scm.node;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.StorageReportProto;
@@ -44,16 +45,19 @@
   private List<StorageReportProto> storageReports;
   private List<MetadataStorageReportProto> metadataStorageReports;
 
+  private NodeStatus nodeStatus;
+
   /**
    * Constructs DatanodeInfo from DatanodeDetails.
    *
    * @param datanodeDetails Details about the datanode
    */
-  public DatanodeInfo(DatanodeDetails datanodeDetails) {
+  public DatanodeInfo(DatanodeDetails datanodeDetails, NodeStatus nodeStatus) {
     super(datanodeDetails);
     this.lock = new ReentrantReadWriteLock();
     this.lastHeartbeatTime = Time.monotonicNow();
     this.storageReports = Collections.emptyList();
+    this.nodeStatus = nodeStatus;
     this.metadataStorageReports = Collections.emptyList();
   }
 
@@ -61,9 +65,20 @@
    * Updates the last heartbeat time with current time.
    */
   public void updateLastHeartbeatTime() {
+    updateLastHeartbeatTime(Time.monotonicNow());
+  }
+
+  /**
+   * Sets the last heartbeat time to a given value. Intended to be used
+   * only for tests.
+   *
+   * @param milliSecondsSinceEpoch - ms since Epoch to set as the heartbeat time
+   */
+  @VisibleForTesting
+  public void updateLastHeartbeatTime(long milliSecondsSinceEpoch) {
     try {
       lock.writeLock().lock();
-      lastHeartbeatTime = Time.monotonicNow();
+      lastHeartbeatTime = milliSecondsSinceEpoch;
     } finally {
       lock.writeLock().unlock();
     }
@@ -171,6 +186,37 @@
     return lastStatsUpdatedTime;
   }
 
+  /**
+   * Return the current NodeStatus for the datanode.
+   *
+   * @return NodeStatus - the current nodeStatus
+   */
+  public NodeStatus getNodeStatus() {
+    try {
+      lock.readLock().lock();
+      return nodeStatus;
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Update the NodeStatus for this datanode. When using this method
+   * be ware of the potential for lost updates if two threads read the
+   * current status, update one field and then write it back without
+   * locking enforced outside of this class.
+   *
+   * @param newNodeStatus - the new NodeStatus object
+   */
+  public void setNodeStatus(NodeStatus newNodeStatus) {
+    try {
+      lock.writeLock().lock();
+      this.nodeStatus = newNodeStatus;
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
   @Override
   public int hashCode() {
     return super.hashCode();
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
index 6a56fc3..b4fc28a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/DeadNodeHandler.java
@@ -78,8 +78,11 @@
       destroyPipelines(datanodeDetails);
       closeContainers(datanodeDetails, publisher);
 
-      // Remove the container replicas associated with the dead node.
-      removeContainerReplicas(datanodeDetails);
+      // Remove the container replicas associated with the dead node unless it
+      // is IN_MAINTENANCE
+      if (!nodeManager.getNodeStatus(datanodeDetails).isInMaintenance()) {
+        removeContainerReplicas(datanodeDetails);
+      }
 
     } catch (NodeNotFoundException ex) {
       // This should not happen, we cannot get a dead node event for an
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidHostStringException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidHostStringException.java
new file mode 100644
index 0000000..c4046c1
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidHostStringException.java
@@ -0,0 +1,34 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown by the NodeDecommissionManager when it encounters
+ * host strings it does not expect or understand.
+ */
+
+public class InvalidHostStringException extends IOException {
+  public InvalidHostStringException(String msg) {
+    super(msg);
+  }
+
+  public InvalidHostStringException(String msg, Exception e) {
+    super(msg, e);
+  }
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidNodeStateException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidNodeStateException.java
new file mode 100644
index 0000000..9c82398
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/InvalidNodeStateException.java
@@ -0,0 +1,34 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import java.io.IOException;
+
+/**
+ * Exception thrown by the NodeDecommissionManager when it encounters
+ * host strings it does not expect or understand.
+ */
+
+public class InvalidNodeStateException extends IOException {
+  public InvalidNodeStateException(String msg) {
+    super(msg);
+  }
+
+  public InvalidNodeStateException(String msg, Exception e) {
+    super(msg, e);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java
index a40a63a..f0f9b72 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NewNodeHandler.java
@@ -20,9 +20,13 @@
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Handles New Node event.
@@ -30,11 +34,16 @@
 public class NewNodeHandler implements EventHandler<DatanodeDetails> {
 
   private final PipelineManager pipelineManager;
+  private final NodeDecommissionManager decommissionManager;
   private final ConfigurationSource conf;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NewNodeHandler.class);
 
   public NewNodeHandler(PipelineManager pipelineManager,
+      NodeDecommissionManager decommissionManager,
       ConfigurationSource conf) {
     this.pipelineManager = pipelineManager;
+    this.decommissionManager = decommissionManager;
     this.conf = conf;
   }
 
@@ -42,5 +51,16 @@
   public void onMessage(DatanodeDetails datanodeDetails,
       EventPublisher publisher) {
     pipelineManager.triggerPipelineCreation();
+    if (datanodeDetails.getPersistedOpState()
+        != HddsProtos.NodeOperationalState.IN_SERVICE) {
+      try {
+        decommissionManager.continueAdminForNode(datanodeDetails);
+      } catch (NodeNotFoundException e) {
+        // Should not happen, as the node has just registered to call this event
+        // handler.
+        LOG.warn("NodeNotFound when adding the node to the decommissionManager",
+            e);
+      }
+    }
   }
 }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java
new file mode 100644
index 0000000..30cae10
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeDecommissionManager.java
@@ -0,0 +1,369 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ReplicationManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.UnknownHostException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Class used to manage datanodes scheduled for maintenance or decommission.
+ */
+public class NodeDecommissionManager {
+
+  private ScheduledExecutorService executor;
+  private DatanodeAdminMonitor monitor;
+
+  private NodeManager nodeManager;
+  //private ContainerManager containerManager;
+  private EventPublisher eventQueue;
+  private ReplicationManager replicationManager;
+  private OzoneConfiguration conf;
+  private boolean useHostnames;
+  private long monitorInterval;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(NodeDecommissionManager.class);
+
+  static class HostDefinition {
+    private String rawHostname;
+    private String hostname;
+    private int port;
+
+    HostDefinition(String hostname) throws InvalidHostStringException {
+      this.rawHostname = hostname;
+      parseHostname();
+    }
+
+    public String getRawHostname() {
+      return rawHostname;
+    }
+
+    public String getHostname() {
+      return hostname;
+    }
+
+    public int getPort() {
+      return port;
+    }
+
+    private void parseHostname() throws InvalidHostStringException{
+      try {
+        // A URI *must* have a scheme, so just create a fake one
+        URI uri = new URI("empty://"+rawHostname.trim());
+        this.hostname = uri.getHost();
+        this.port = uri.getPort();
+
+        if (this.hostname == null) {
+          throw new InvalidHostStringException("The string "+rawHostname+
+              " does not contain a value hostname or hostname:port definition");
+        }
+      } catch (URISyntaxException e) {
+        throw new InvalidHostStringException(
+            "Unable to parse the hoststring "+rawHostname, e);
+      }
+    }
+  }
+
+  private List<DatanodeDetails> mapHostnamesToDatanodes(List<String> hosts)
+      throws InvalidHostStringException {
+    List<DatanodeDetails> results = new LinkedList<>();
+    for (String hostString : hosts) {
+      HostDefinition host = new HostDefinition(hostString);
+      InetAddress addr;
+      try {
+        addr = InetAddress.getByName(host.getHostname());
+      } catch (UnknownHostException e) {
+        throw new InvalidHostStringException("Unable to resolve the host "
+            +host.getRawHostname(), e);
+      }
+      String dnsName;
+      if (useHostnames) {
+        dnsName = addr.getHostName();
+      } else {
+        dnsName = addr.getHostAddress();
+      }
+      List<DatanodeDetails> found = nodeManager.getNodesByAddress(dnsName);
+      if (found.size() == 0) {
+        throw new InvalidHostStringException("The string " +
+            host.getRawHostname()+" resolved to "+dnsName +
+            " is not found in SCM");
+      } else if (found.size() == 1) {
+        if (host.getPort() != -1 &&
+            !validateDNPortMatch(host.getPort(), found.get(0))) {
+          throw new InvalidHostStringException("The string "+
+              host.getRawHostname()+" matched a single datanode, but the "+
+              "given port is not used by that Datanode");
+        }
+        results.add(found.get(0));
+      } else if (found.size() > 1) {
+        DatanodeDetails match = null;
+        for(DatanodeDetails dn : found) {
+          if (validateDNPortMatch(host.getPort(), dn)) {
+            match = dn;
+            break;
+          }
+        }
+        if (match == null) {
+          throw new InvalidHostStringException("The string " +
+              host.getRawHostname()+ "matched multiple Datanodes, but no "+
+              "datanode port matched the given port");
+        }
+        results.add(match);
+      }
+    }
+    return results;
+  }
+
+  /**
+   * Check if the passed port is used by the given DatanodeDetails object. If
+   * it is, return true, otherwise return false.
+   * @param port Port number to check if it is used by the datanode
+   * @param dn Datanode to check if it is using the given port
+   * @return True if port is used by the datanode. False otherwise.
+   */
+  private boolean validateDNPortMatch(int port, DatanodeDetails dn) {
+    for (DatanodeDetails.Port p : dn.getPorts()) {
+      if (p.getValue() == port) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  public NodeDecommissionManager(OzoneConfiguration config, NodeManager nm,
+      ContainerManager containerManager,
+      EventPublisher eventQueue, ReplicationManager rm) {
+    this.nodeManager = nm;
+    conf = config;
+    //this.containerManager = containerManager;
+    this.eventQueue = eventQueue;
+    this.replicationManager = rm;
+
+    executor = Executors.newScheduledThreadPool(1,
+        new ThreadFactoryBuilder().setNameFormat("DatanodeAdminManager-%d")
+            .setDaemon(true).build());
+
+    useHostnames = conf.getBoolean(
+        DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
+        DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
+
+    monitorInterval = conf.getTimeDuration(
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL,
+        ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL_DEFAULT,
+        TimeUnit.SECONDS);
+    if (monitorInterval <= 0) {
+      LOG.warn("{} must be greater than zero, defaulting to {}",
+          ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL,
+          ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL_DEFAULT);
+      conf.set(ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL,
+          ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL_DEFAULT);
+      monitorInterval = conf.getTimeDuration(
+          ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL,
+          ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL_DEFAULT,
+          TimeUnit.SECONDS);
+    }
+
+    monitor = new DatanodeAdminMonitorImpl(conf, eventQueue, nodeManager,
+        replicationManager);
+
+    executor.scheduleAtFixedRate(monitor, monitorInterval, monitorInterval,
+        TimeUnit.SECONDS);
+  }
+
+  @VisibleForTesting
+  public DatanodeAdminMonitor getMonitor() {
+    return monitor;
+  }
+
+  public synchronized void decommissionNodes(List nodes)
+      throws InvalidHostStringException {
+    List<DatanodeDetails> dns = mapHostnamesToDatanodes(nodes);
+    for (DatanodeDetails dn : dns) {
+      try {
+        startDecommission(dn);
+      } catch (NodeNotFoundException e) {
+        // We already validated the host strings and retrieved the DnDetails
+        // object from the node manager. Therefore we should never get a
+        // NodeNotFoundException here expect if the node is remove in the
+        // very short window between validation and starting decom. Therefore
+        // log a warning and ignore the exception
+        LOG.warn("The host {} was not found in SCM. Ignoring the request to "+
+            "decommission it", dn.getHostName());
+      } catch (InvalidNodeStateException e) {
+        // TODO - decide how to handle this. We may not want to fail all nodes
+        //        only one is in a bad state, as some nodes may have been OK
+        //        and already processed. Perhaps we should return a list of
+        //        error and feed that all the way back to the client?
+      }
+    }
+  }
+
+  /**
+   * If a SCM is restarted, then upon re-registration the datanode will already
+   * be in DECOMMISSIONING or ENTERING_MAINTENANCE state. In that case, it
+   * needs to be added back into the monitor to track its progress.
+   * @param dn Datanode to add back to tracking.
+   * @throws NodeNotFoundException
+   */
+  public synchronized void continueAdminForNode(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    NodeOperationalState opState = getNodeStatus(dn).getOperationalState();
+    if (opState == NodeOperationalState.DECOMMISSIONING
+        || opState == NodeOperationalState.ENTERING_MAINTENANCE
+        || opState == NodeOperationalState.IN_MAINTENANCE) {
+      monitor.startMonitoring(dn);
+    }
+  }
+
+  public synchronized void startDecommission(DatanodeDetails dn)
+      throws NodeNotFoundException, InvalidNodeStateException {
+    NodeStatus nodeStatus = getNodeStatus(dn);
+    NodeOperationalState opState = nodeStatus.getOperationalState();
+    if (opState == NodeOperationalState.IN_SERVICE) {
+      LOG.info("Starting Decommission for node {}", dn);
+      nodeManager.setNodeOperationalState(
+          dn, NodeOperationalState.DECOMMISSIONING);
+      monitor.startMonitoring(dn);
+    } else if (nodeStatus.isDecommission()) {
+      LOG.info("Start Decommission called on node {} in state {}. Nothing to "+
+          "do.", dn, opState);
+    } else {
+      LOG.error("Cannot decommission node {} in state {}", dn, opState);
+      throw new InvalidNodeStateException("Cannot decommission node "+
+          dn +" in state "+ opState);
+    }
+  }
+
+  public synchronized void recommissionNodes(List nodes)
+      throws InvalidHostStringException {
+    List<DatanodeDetails> dns = mapHostnamesToDatanodes(nodes);
+    for (DatanodeDetails dn : dns) {
+      try {
+        recommission(dn);
+      } catch (NodeNotFoundException e) {
+        // We already validated the host strings and retrieved the DnDetails
+        // object from the node manager. Therefore we should never get a
+        // NodeNotFoundException here expect if the node is remove in the
+        // very short window between validation and starting decom. Therefore
+        // log a warning and ignore the exception
+        LOG.warn("The host {} was not found in SCM. Ignoring the request to "+
+            "recommission it", dn.getHostName());
+      }
+    }
+  }
+
+  public synchronized void recommission(DatanodeDetails dn)
+      throws NodeNotFoundException{
+    NodeStatus nodeStatus = getNodeStatus(dn);
+    NodeOperationalState opState = nodeStatus.getOperationalState();
+    if (opState != NodeOperationalState.IN_SERVICE) {
+      // The node will be set back to IN_SERVICE when it is processed by the
+      // monitor
+      monitor.stopMonitoring(dn);
+      LOG.info("Queued node {} for recommission", dn);
+    } else {
+      LOG.info("Recommission called on node {} with state {}. "+
+          "Nothing to do.", dn, opState);
+    }
+  }
+
+  public synchronized void startMaintenanceNodes(List nodes, int endInHours)
+      throws InvalidHostStringException {
+    List<DatanodeDetails> dns = mapHostnamesToDatanodes(nodes);
+    for (DatanodeDetails dn : dns) {
+      try {
+        startMaintenance(dn, endInHours);
+      } catch (NodeNotFoundException e) {
+        // We already validated the host strings and retrieved the DnDetails
+        // object from the node manager. Therefore we should never get a
+        // NodeNotFoundException here expect if the node is remove in the
+        // very short window between validation and starting decom. Therefore
+        // log a warning and ignore the exception
+        LOG.warn("The host {} was not found in SCM. Ignoring the request to "+
+            "start maintenance on it", dn.getHostName());
+      } catch (InvalidNodeStateException e) {
+        // TODO - decide how to handle this. We may not want to fail all nodes
+        //        only one is in a bad state, as some nodes may have been OK
+        //        and already processed. Perhaps we should return a list of
+        //        error and feed that all the way back to the client?
+      }
+    }
+  }
+
+  // TODO - If startMaintenance is called on a host already in maintenance,
+  //        then we should update the end time?
+  public synchronized void startMaintenance(DatanodeDetails dn, int endInHours)
+      throws NodeNotFoundException, InvalidNodeStateException {
+    NodeStatus nodeStatus = getNodeStatus(dn);
+    NodeOperationalState opState = nodeStatus.getOperationalState();
+
+    long maintenanceEnd = 0;
+    if (endInHours != 0) {
+      maintenanceEnd =
+          (System.currentTimeMillis() / 1000L) + (endInHours * 60L * 60L);
+    }
+    if (opState == NodeOperationalState.IN_SERVICE) {
+      nodeManager.setNodeOperationalState(
+          dn, NodeOperationalState.ENTERING_MAINTENANCE, maintenanceEnd);
+      monitor.startMonitoring(dn);
+      LOG.info("Starting Maintenance for node {}", dn);
+    } else if (nodeStatus.isMaintenance()) {
+      LOG.info("Starting Maintenance called on node {} with state {}. "+
+          "Nothing to do.", dn, opState);
+    } else {
+      LOG.error("Cannot start maintenance on node {} in state {}", dn, opState);
+      throw new InvalidNodeStateException("Cannot start maintenance on node "+
+          dn +" in state "+ opState);
+    }
+  }
+
+  /**
+   *  Stops the decommission monitor from running when SCM is shutdown.
+   */
+  public void stop() {
+    if (executor != null) {
+      executor.shutdown();
+    }
+  }
+
+  private NodeStatus getNodeStatus(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    return nodeManager.getNodeStatus(dn);
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java
index 4af2357..22a1e81 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManager.java
@@ -27,6 +27,7 @@
 import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.ozone.protocol.StorageContainerNodeProtocol;
 import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
@@ -64,18 +65,38 @@
     EventHandler<CommandForDatanode>, NodeManagerMXBean, Closeable {
 
   /**
-   * Gets all Live Datanodes that is currently communicating with SCM.
-   * @param nodeState - State of the node
+   * Gets all Live Datanodes that are currently communicating with SCM.
+   * @param nodeStatus - Status of the node to return
    * @return List of Datanodes that are Heartbeating SCM.
    */
-  List<DatanodeDetails> getNodes(NodeState nodeState);
+  List<DatanodeDetails> getNodes(NodeStatus nodeStatus);
 
   /**
-   * Returns the Number of Datanodes that are communicating with SCM.
-   * @param nodeState - State of the node
+   * Gets all Live Datanodes that is currently communicating with SCM.
+   * @param opState - The operational state of the node
+   * @param health - The health of the node
+   * @return List of Datanodes that are Heartbeating SCM.
+   */
+  List<DatanodeDetails> getNodes(
+      NodeOperationalState opState, NodeState health);
+
+  /**
+   * Returns the Number of Datanodes that are communicating with SCM with the
+   * given status.
+   * @param nodeStatus - State of the node
    * @return int -- count
    */
-  int getNodeCount(NodeState nodeState);
+  int getNodeCount(NodeStatus nodeStatus);
+
+  /**
+   * Returns the Number of Datanodes that are communicating with SCM in the
+   * given state.
+   * @param opState - The operational state of the node
+   * @param health - The health of the node
+   * @return int -- count
+   */
+  int getNodeCount(
+      NodeOperationalState opState, NodeState health);
 
   /**
    * Get all datanodes known to SCM.
@@ -105,11 +126,33 @@
   SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails);
 
   /**
-   * Returns the node state of a specific node.
+   * Returns the node status of a specific node.
    * @param datanodeDetails DatanodeDetails
-   * @return Healthy/Stale/Dead.
+   * @return NodeStatus for the node
+   * @throws NodeNotFoundException if the node does not exist
    */
-  NodeState getNodeState(DatanodeDetails datanodeDetails);
+  NodeStatus getNodeStatus(DatanodeDetails datanodeDetails)
+      throws NodeNotFoundException;
+
+  /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  void setNodeOperationalState(DatanodeDetails datanodeDetails,
+      NodeOperationalState newState) throws NodeNotFoundException;
+
+  /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   * @param opStateExpiryEpocSec Seconds from the epoch when the operational
+   *                             state should end. Zero indicates the state
+   *                             never end.
+   */
+  void setNodeOperationalState(DatanodeDetails datanodeDetails,
+       NodeOperationalState newState,
+       long opStateExpiryEpocSec) throws NodeNotFoundException;
 
   /**
    * Get set of pipelines a datanode is part of.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManagerMXBean.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManagerMXBean.java
index 3e7ecf7..c1c6d0d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManagerMXBean.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeManagerMXBean.java
@@ -34,7 +34,7 @@
    *
    * @return A state to number of nodes that in this state mapping
    */
-  Map<String, Integer> getNodeCount();
+  Map<String, Map<String, Integer>> getNodeCount();
 
   /**
    * Get the disk metrics like capacity, usage and remaining based on the
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java
index d51961f..d7af7f5 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStateManager.java
@@ -19,7 +19,6 @@
 package org.apache.hadoop.hdds.scm.node;
 
 import java.io.Closeable;
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -33,7 +32,9 @@
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.node.states.Node2PipelineMap;
@@ -79,7 +80,7 @@
    * Node's life cycle events.
    */
   private enum NodeLifeCycleEvent {
-    TIMEOUT, RESTORE, RESURRECT, DECOMMISSION, DECOMMISSIONED
+    TIMEOUT, RESTORE, RESURRECT
   }
 
   private static final Logger LOG = LoggerFactory
@@ -89,7 +90,7 @@
   /**
    * StateMachine for node lifecycle.
    */
-  private final StateMachine<NodeState, NodeLifeCycleEvent> stateMachine;
+  private final StateMachine<NodeState, NodeLifeCycleEvent> nodeHealthSM;
   /**
    * This is the map which maintains the current state of all datanodes.
    */
@@ -156,9 +157,8 @@
     this.state2EventMap = new HashMap<>();
     initialiseState2EventMap();
     Set<NodeState> finalStates = new HashSet<>();
-    finalStates.add(NodeState.DECOMMISSIONED);
-    this.stateMachine = new StateMachine<>(NodeState.HEALTHY, finalStates);
-    initializeStateMachine();
+    this.nodeHealthSM = new StateMachine<>(NodeState.HEALTHY, finalStates);
+    initializeStateMachines();
     heartbeatCheckerIntervalMs = HddsServerUtil
         .getScmheartbeatCheckerInterval(conf);
     staleNodeIntervalMs = HddsServerUtil.getStaleNodeInterval(conf);
@@ -184,8 +184,8 @@
   private void initialiseState2EventMap() {
     state2EventMap.put(NodeState.STALE, SCMEvents.STALE_NODE);
     state2EventMap.put(NodeState.DEAD, SCMEvents.DEAD_NODE);
-    state2EventMap
-        .put(NodeState.HEALTHY, SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE);
+    state2EventMap.put(NodeState.HEALTHY,
+        SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE);
   }
 
   /*
@@ -204,18 +204,6 @@
    * State: DEAD            -------------------> HEALTHY
    * Event:                       RESURRECT
    *
-   * State: HEALTHY         -------------------> DECOMMISSIONING
-   * Event:                     DECOMMISSION
-   *
-   * State: STALE           -------------------> DECOMMISSIONING
-   * Event:                     DECOMMISSION
-   *
-   * State: DEAD            -------------------> DECOMMISSIONING
-   * Event:                     DECOMMISSION
-   *
-   * State: DECOMMISSIONING -------------------> DECOMMISSIONED
-   * Event:                     DECOMMISSIONED
-   *
    *  Node State Flow
    *
    *  +--------------------------------------------------------+
@@ -225,47 +213,21 @@
    *  |   |                          |                         |
    *  V   V                          |                         |
    * [HEALTHY]------------------->[STALE]------------------->[DEAD]
-   *    |         (TIMEOUT)          |         (TIMEOUT)       |
-   *    |                            |                         |
-   *    |                            |                         |
-   *    |                            |                         |
-   *    |                            |                         |
-   *    | (DECOMMISSION)             | (DECOMMISSION)          | (DECOMMISSION)
-   *    |                            V                         |
-   *    +------------------->[DECOMMISSIONING]<----------------+
-   *                                 |
-   *                                 | (DECOMMISSIONED)
-   *                                 |
-   *                                 V
-   *                          [DECOMMISSIONED]
    *
    */
 
   /**
    * Initializes the lifecycle of node state machine.
    */
-  private void initializeStateMachine() {
-    stateMachine.addTransition(
+  private void initializeStateMachines() {
+    nodeHealthSM.addTransition(
         NodeState.HEALTHY, NodeState.STALE, NodeLifeCycleEvent.TIMEOUT);
-    stateMachine.addTransition(
+    nodeHealthSM.addTransition(
         NodeState.STALE, NodeState.DEAD, NodeLifeCycleEvent.TIMEOUT);
-    stateMachine.addTransition(
+    nodeHealthSM.addTransition(
         NodeState.STALE, NodeState.HEALTHY, NodeLifeCycleEvent.RESTORE);
-    stateMachine.addTransition(
+    nodeHealthSM.addTransition(
         NodeState.DEAD, NodeState.HEALTHY, NodeLifeCycleEvent.RESURRECT);
-    stateMachine.addTransition(
-        NodeState.HEALTHY, NodeState.DECOMMISSIONING,
-        NodeLifeCycleEvent.DECOMMISSION);
-    stateMachine.addTransition(
-        NodeState.STALE, NodeState.DECOMMISSIONING,
-        NodeLifeCycleEvent.DECOMMISSION);
-    stateMachine.addTransition(
-        NodeState.DEAD, NodeState.DECOMMISSIONING,
-        NodeLifeCycleEvent.DECOMMISSION);
-    stateMachine.addTransition(
-        NodeState.DECOMMISSIONING, NodeState.DECOMMISSIONED,
-        NodeLifeCycleEvent.DECOMMISSIONED);
-
   }
 
   /**
@@ -277,11 +239,33 @@
    */
   public void addNode(DatanodeDetails datanodeDetails)
       throws NodeAlreadyExistsException {
-    nodeStateMap.addNode(datanodeDetails, stateMachine.getInitialState());
+    NodeStatus newNodeStatus = newNodeStatus(datanodeDetails);
+    nodeStateMap.addNode(datanodeDetails, newNodeStatus);
     eventPublisher.fireEvent(SCMEvents.NEW_NODE, datanodeDetails);
   }
 
   /**
+   * When a node registers with SCM, the operational state stored on the
+   * datanode is the source of truth. Therefore, if the datanode reports
+   * anything other than IN_SERVICE on registration, the state in SCM should be
+   * updated to reflect the datanode state.
+   * @param dn DatanodeDetails reported by the datanode
+   */
+  private NodeStatus newNodeStatus(DatanodeDetails dn) {
+    HddsProtos.NodeOperationalState dnOpState = dn.getPersistedOpState();
+    if (dnOpState != NodeOperationalState.IN_SERVICE) {
+      LOG.info("Updating nodeOperationalState on registration as the " +
+              "datanode has a persisted state of {} and expiry of {}",
+          dnOpState, dn.getPersistedOpStateExpiryEpochSec());
+      return new NodeStatus(dnOpState, nodeHealthSM.getInitialState(),
+          dn.getPersistedOpStateExpiryEpochSec());
+    } else {
+      return new NodeStatus(
+          NodeOperationalState.IN_SERVICE, nodeHealthSM.getInitialState());
+    }
+  }
+
+  /**
    * Adds a pipeline in the node2PipelineMap.
    * @param pipeline - Pipeline to be added
    */
@@ -332,59 +316,63 @@
    *
    * @throws NodeNotFoundException if the node is not present
    */
-  public NodeState getNodeState(DatanodeDetails datanodeDetails)
+  public NodeStatus getNodeStatus(DatanodeDetails datanodeDetails)
       throws NodeNotFoundException {
-    return nodeStateMap.getNodeState(datanodeDetails.getUuid());
+    return nodeStateMap.getNodeStatus(datanodeDetails.getUuid());
   }
 
   /**
-   * Returns all the node which are in healthy state.
+   * Returns all the node which are in healthy state, ignoring the operational
+   * state.
    *
    * @return list of healthy nodes
    */
   public List<DatanodeInfo> getHealthyNodes() {
-    return getNodes(NodeState.HEALTHY);
+    return getNodes(null, NodeState.HEALTHY);
   }
 
   /**
-   * Returns all the node which are in stale state.
+   * Returns all the node which are in stale state, ignoring the operational
+   * state.
    *
    * @return list of stale nodes
    */
   public List<DatanodeInfo> getStaleNodes() {
-    return getNodes(NodeState.STALE);
+    return getNodes(null, NodeState.STALE);
   }
 
   /**
-   * Returns all the node which are in dead state.
+   * Returns all the node which are in dead state, ignoring the operational
+   * state.
    *
    * @return list of dead nodes
    */
   public List<DatanodeInfo> getDeadNodes() {
-    return getNodes(NodeState.DEAD);
+    return getNodes(null, NodeState.DEAD);
   }
 
   /**
-   * Returns all the node which are in the specified state.
+   * Returns all the nodes with the specified status.
    *
-   * @param state NodeState
+   * @param status NodeStatus
    *
    * @return list of nodes
    */
-  public List<DatanodeInfo> getNodes(NodeState state) {
-    List<DatanodeInfo> nodes = new ArrayList<>();
-    nodeStateMap.getNodes(state).forEach(
-        uuid -> {
-          try {
-            nodes.add(nodeStateMap.getNodeInfo(uuid));
-          } catch (NodeNotFoundException e) {
-            // This should not happen unless someone else other than
-            // NodeStateManager is directly modifying NodeStateMap and removed
-            // the node entry after we got the list of UUIDs.
-            LOG.error("Inconsistent NodeStateMap! {}", nodeStateMap);
-          }
-        });
-    return nodes;
+  public List<DatanodeInfo> getNodes(NodeStatus status) {
+    return nodeStateMap.getDatanodeInfos(status);
+  }
+
+  /**
+   * Returns all the nodes with the specified operationalState and health.
+   *
+   * @param opState The operationalState of the node
+   * @param health  The node health
+   *
+   * @return list of nodes matching the passed states
+   */
+  public List<DatanodeInfo> getNodes(
+      NodeOperationalState opState, NodeState health) {
+    return nodeStateMap.getDatanodeInfos(opState, health);
   }
 
   /**
@@ -393,19 +381,52 @@
    * @return all the managed nodes
    */
   public List<DatanodeInfo> getAllNodes() {
-    List<DatanodeInfo> nodes = new ArrayList<>();
-    nodeStateMap.getAllNodes().forEach(
-        uuid -> {
-          try {
-            nodes.add(nodeStateMap.getNodeInfo(uuid));
-          } catch (NodeNotFoundException e) {
-            // This should not happen unless someone else other than
-            // NodeStateManager is directly modifying NodeStateMap and removed
-            // the node entry after we got the list of UUIDs.
-            LOG.error("Inconsistent NodeStateMap! {}", nodeStateMap);
-          }
-        });
-    return nodes;
+    return nodeStateMap.getAllDatanodeInfos();
+  }
+
+  /**
+   * Sets the operational state of the given node. Intended to be called when
+   * a node is being decommissioned etc.
+   *
+   * @param dn The datanode having its state set
+   * @param newState The new operational State of the node.
+   */
+  public void setNodeOperationalState(DatanodeDetails dn,
+      NodeOperationalState newState)  throws NodeNotFoundException {
+    setNodeOperationalState(dn, newState, 0);
+  }
+
+  /**
+   * Sets the operational state of the given node. Intended to be called when
+   * a node is being decommissioned etc.
+   *
+   * @param dn The datanode having its state set
+   * @param newState The new operational State of the node.
+   * @param stateExpiryEpochSec The number of seconds from the epoch when the
+   *                            operational state should expire. Passing zero
+   *                            indicates the state will never expire
+   */
+  public void setNodeOperationalState(DatanodeDetails dn,
+      NodeOperationalState newState,
+      long stateExpiryEpochSec)  throws NodeNotFoundException {
+    DatanodeInfo dni = nodeStateMap.getNodeInfo(dn.getUuid());
+    NodeStatus oldStatus = dni.getNodeStatus();
+    if (oldStatus.getOperationalState() != newState ||
+        oldStatus.getOpStateExpiryEpochSeconds() != stateExpiryEpochSec) {
+      nodeStateMap.updateNodeOperationalState(
+          dn.getUuid(), newState, stateExpiryEpochSec);
+      // This will trigger an event based on the nodes health when the
+      // operational state changes. Eg a node that was IN_MAINTENANCE goes
+      // to IN_SERVICE + HEALTHY. This will trigger the HEALTHY node event to
+      // create new pipelines. OTH, if the nodes goes IN_MAINTENANCE to
+      // IN_SERVICE + DEAD, it will trigger the dead node handler to remove its
+      // container replicas. Sometimes the event will do nothing, but it will
+      // not do any harm either. Eg DECOMMISSIONING -> DECOMMISSIONED + HEALTHY
+      // but the pipeline creation logic will ignore decommissioning nodes.
+      if (oldStatus.getOperationalState() != newState) {
+        fireHealthStateEvent(oldStatus.getHealth(), dn);
+      }
+    }
   }
 
   /**
@@ -418,41 +439,53 @@
   }
 
   /**
-   * Returns the count of healthy nodes.
+   * Returns the count of healthy nodes, ignoring operational state.
    *
    * @return healthy node count
    */
   public int getHealthyNodeCount() {
-    return getNodeCount(NodeState.HEALTHY);
+    return getHealthyNodes().size();
   }
 
   /**
-   * Returns the count of stale nodes.
+   * Returns the count of stale nodes, ignoring operational state.
    *
    * @return stale node count
    */
   public int getStaleNodeCount() {
-    return getNodeCount(NodeState.STALE);
+    return getStaleNodes().size();
   }
 
   /**
-   * Returns the count of dead nodes.
+   * Returns the count of dead nodes, ignoring operational state.
    *
    * @return dead node count
    */
   public int getDeadNodeCount() {
-    return getNodeCount(NodeState.DEAD);
+    return getDeadNodes().size();
   }
 
   /**
-   * Returns the count of nodes in specified state.
+   * Returns the count of nodes in specified status.
    *
-   * @param state NodeState
+   * @param status NodeState
    *
    * @return node count
    */
-  public int getNodeCount(NodeState state) {
-    return nodeStateMap.getNodeCount(state);
+  public int getNodeCount(NodeStatus status) {
+    return nodeStateMap.getNodeCount(status);
+  }
+
+  /**
+   * Returns the count of nodes in the specified states.
+   *
+   * @param opState The operational state of the node
+   * @param health The health of the node
+   *
+   * @return node count
+   */
+  public int getNodeCount(NodeOperationalState opState, NodeState health) {
+    return nodeStateMap.getNodeCount(opState, health);
   }
 
   /**
@@ -551,7 +584,8 @@
     scheduleNextHealthCheck();
   }
 
-  private void checkNodesHealth() {
+  @VisibleForTesting
+  public void checkNodesHealth() {
 
     /*
      *
@@ -593,39 +627,32 @@
     Predicate<Long> deadNodeCondition =
         (lastHbTime) -> lastHbTime < staleNodeDeadline;
     try {
-      for (NodeState state : NodeState.values()) {
-        List<UUID> nodes = nodeStateMap.getNodes(state);
-        for (UUID id : nodes) {
-          DatanodeInfo node = nodeStateMap.getNodeInfo(id);
-          switch (state) {
-          case HEALTHY:
-            // Move the node to STALE if the last heartbeat time is less than
-            // configured stale-node interval.
-            updateNodeState(node, staleNodeCondition, state,
-                NodeLifeCycleEvent.TIMEOUT);
-            break;
-          case STALE:
-            // Move the node to DEAD if the last heartbeat time is less than
-            // configured dead-node interval.
-            updateNodeState(node, deadNodeCondition, state,
-                NodeLifeCycleEvent.TIMEOUT);
-            // Restore the node if we have received heartbeat before configured
-            // stale-node interval.
-            updateNodeState(node, healthyNodeCondition, state,
-                NodeLifeCycleEvent.RESTORE);
-            break;
-          case DEAD:
-            // Resurrect the node if we have received heartbeat before
-            // configured stale-node interval.
-            updateNodeState(node, healthyNodeCondition, state,
-                NodeLifeCycleEvent.RESURRECT);
-            break;
-          // We don't do anything for DECOMMISSIONING and DECOMMISSIONED in
-          // heartbeat processing.
-          case DECOMMISSIONING:
-          case DECOMMISSIONED:
-          default:
-          }
+      for(DatanodeInfo node : nodeStateMap.getAllDatanodeInfos()) {
+        NodeStatus status = nodeStateMap.getNodeStatus(node.getUuid());
+        switch (status.getHealth()) {
+        case HEALTHY:
+          // Move the node to STALE if the last heartbeat time is less than
+          // configured stale-node interval.
+          updateNodeState(node, staleNodeCondition, status,
+              NodeLifeCycleEvent.TIMEOUT);
+          break;
+        case STALE:
+          // Move the node to DEAD if the last heartbeat time is less than
+          // configured dead-node interval.
+          updateNodeState(node, deadNodeCondition, status,
+              NodeLifeCycleEvent.TIMEOUT);
+          // Restore the node if we have received heartbeat before configured
+          // stale-node interval.
+          updateNodeState(node, healthyNodeCondition, status,
+              NodeLifeCycleEvent.RESTORE);
+          break;
+        case DEAD:
+          // Resurrect the node if we have received heartbeat before
+          // configured stale-node interval.
+          updateNodeState(node, healthyNodeCondition, status,
+              NodeLifeCycleEvent.RESURRECT);
+          break;
+        default:
         }
       }
     } catch (NodeNotFoundException e) {
@@ -684,27 +711,35 @@
    *
    * @param node DatanodeInfo
    * @param condition condition to check
-   * @param state current state of node
+   * @param status current status of node
    * @param lifeCycleEvent NodeLifeCycleEvent to be applied if condition
    *                       matches
    *
    * @throws NodeNotFoundException if the node is not present
    */
   private void updateNodeState(DatanodeInfo node, Predicate<Long> condition,
-      NodeState state, NodeLifeCycleEvent lifeCycleEvent)
+      NodeStatus status, NodeLifeCycleEvent lifeCycleEvent)
       throws NodeNotFoundException {
     try {
       if (condition.test(node.getLastHeartbeatTime())) {
-        NodeState newState = stateMachine.getNextState(state, lifeCycleEvent);
-        nodeStateMap.updateNodeState(node.getUuid(), state, newState);
-        if (state2EventMap.containsKey(newState)) {
-          eventPublisher.fireEvent(state2EventMap.get(newState), node);
-        }
+        NodeState newHealthState = nodeHealthSM.
+            getNextState(status.getHealth(), lifeCycleEvent);
+        NodeStatus newStatus =
+            nodeStateMap.updateNodeHealthState(node.getUuid(), newHealthState);
+        fireHealthStateEvent(newStatus.getHealth(), node);
       }
     } catch (InvalidStateTransitionException e) {
       LOG.warn("Invalid state transition of node {}." +
               " Current state: {}, life cycle event: {}",
-          node, state, lifeCycleEvent);
+          node, status.getHealth(), lifeCycleEvent);
+    }
+  }
+
+  private void fireHealthStateEvent(HddsProtos.NodeState health,
+      DatanodeDetails node) {
+    Event<DatanodeDetails> event = state2EventMap.get(health);
+    if (event != null) {
+      eventPublisher.fireEvent(event, node);
     }
   }
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStatus.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStatus.java
new file mode 100644
index 0000000..72ca015
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/NodeStatus.java
@@ -0,0 +1,206 @@
+/**
+ * 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.node;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+
+import java.util.Objects;
+
+/**
+ * This class is used to capture the current status of a datanode. This
+ * includes its health (healthy, stale or dead) and its operation status (
+ * in_service, decommissioned and maintenance mode) along with the expiry time
+ * for the operational state (used with maintenance mode).
+ */
+public class NodeStatus {
+
+  private HddsProtos.NodeOperationalState operationalState;
+  private HddsProtos.NodeState health;
+  private long opStateExpiryEpochSeconds;
+
+  public NodeStatus(HddsProtos.NodeOperationalState operationalState,
+             HddsProtos.NodeState health) {
+    this.operationalState = operationalState;
+    this.health = health;
+    this.opStateExpiryEpochSeconds = 0;
+  }
+
+  public NodeStatus(HddsProtos.NodeOperationalState operationalState,
+                    HddsProtos.NodeState health,
+                    long opStateExpireEpocSeconds) {
+    this.operationalState = operationalState;
+    this.health = health;
+    this.opStateExpiryEpochSeconds = opStateExpireEpocSeconds;
+  }
+
+  public static NodeStatus inServiceHealthy() {
+    return new NodeStatus(HddsProtos.NodeOperationalState.IN_SERVICE,
+        HddsProtos.NodeState.HEALTHY);
+  }
+
+  public static NodeStatus inServiceStale() {
+    return new NodeStatus(HddsProtos.NodeOperationalState.IN_SERVICE,
+        HddsProtos.NodeState.STALE);
+  }
+
+  public static NodeStatus inServiceDead() {
+    return new NodeStatus(HddsProtos.NodeOperationalState.IN_SERVICE,
+        HddsProtos.NodeState.DEAD);
+  }
+
+  public HddsProtos.NodeState getHealth() {
+    return health;
+  }
+
+  public HddsProtos.NodeOperationalState getOperationalState() {
+    return operationalState;
+  }
+
+  public long getOpStateExpiryEpochSeconds() {
+    return opStateExpiryEpochSeconds;
+  }
+
+  public boolean operationalStateExpired() {
+    if (0 == opStateExpiryEpochSeconds) {
+      return false;
+    }
+    return System.currentTimeMillis() / 1000 >= opStateExpiryEpochSeconds;
+  }
+
+  /**
+   * Returns true if the nodeStatus indicates the node is in any decommission
+   * state.
+   *
+   * @return True if the node is in any decommission state, false otherwise
+   */
+  public boolean isDecommission() {
+    return operationalState == HddsProtos.NodeOperationalState.DECOMMISSIONING
+        || operationalState == HddsProtos.NodeOperationalState.DECOMMISSIONED;
+  }
+
+  /**
+   * Returns true if the node is currently decommissioning.
+   *
+   * @return True if the node is decommissioning, false otherwise
+   */
+  public boolean isDecommissioning() {
+    return operationalState == HddsProtos.NodeOperationalState.DECOMMISSIONING;
+  }
+
+  /**
+   * Returns true if the node is decommissioned.
+   *
+   * @return True if the node is decommissioned, false otherwise
+   */
+  public boolean isDecommissioned() {
+    return operationalState == HddsProtos.NodeOperationalState.DECOMMISSIONED;
+  }
+
+  /**
+   * Returns true if the node is in any maintenance state.
+   *
+   * @return True if the node is in any maintenance state, false otherwise
+   */
+  public boolean isMaintenance() {
+    return operationalState
+        == HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE
+        || operationalState == HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+  }
+
+  /**
+   * Returns true if the node is currently entering maintenance.
+   *
+   * @return True if the node is entering maintenance, false otherwise
+   */
+  public boolean isEnteringMaintenance() {
+    return operationalState
+        == HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE;
+  }
+
+  /**
+   * Returns true if the node is currently in maintenance.
+   *
+   * @return True if the node is in maintenance, false otherwise.
+   */
+  public boolean isInMaintenance() {
+    return operationalState == HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+  }
+
+  /**
+   * Returns true if the nodeStatus is healthy (ie not stale or dead) and false
+   * otherwise.
+   *
+   * @return True if the node is Healthy, false otherwise
+   */
+  public boolean isHealthy() {
+    return health == HddsProtos.NodeState.HEALTHY;
+  }
+
+  /**
+   * Returns true if the nodeStatus is either healthy or stale and false
+   * otherwise.
+   *
+   * @return True is the node is Healthy or Stale, false otherwise.
+   */
+  public boolean isAlive() {
+    return health == HddsProtos.NodeState.HEALTHY
+        || health == HddsProtos.NodeState.STALE;
+  }
+
+  /**
+   * Returns true if the nodeStatus is dead and false otherwise.
+   *
+   * @return True is the node is Dead, false otherwise.
+   */
+  public boolean isDead() {
+    return health == HddsProtos.NodeState.DEAD;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    NodeStatus other = (NodeStatus) obj;
+    if (this.operationalState == other.operationalState &&
+        this.health == other.health
+        && this.opStateExpiryEpochSeconds == other.opStateExpiryEpochSeconds) {
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(health, operationalState, opStateExpiryEpochSeconds);
+  }
+
+  @Override
+  public String toString() {
+    return "OperationalState: "+operationalState+" Health: "+health+
+        " OperastionStateExpiry: "+opStateExpiryEpochSeconds;
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index 328f271..323d79a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -36,6 +36,7 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
@@ -64,11 +65,13 @@
 import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
 import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
 import org.apache.hadoop.util.ReflectionUtils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import org.apache.hadoop.util.Time;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -161,9 +164,29 @@
    * @return List of Datanodes that are known to SCM in the requested state.
    */
   @Override
-  public List<DatanodeDetails> getNodes(NodeState nodestate) {
-    return nodeStateManager.getNodes(nodestate).stream()
-        .map(node -> (DatanodeDetails) node).collect(Collectors.toList());
+  public List<DatanodeDetails> getNodes(NodeStatus nodeStatus) {
+    return nodeStateManager.getNodes(nodeStatus)
+        .stream()
+        .map(node -> (DatanodeDetails)node).collect(Collectors.toList());
+  }
+
+  /**
+   * Returns all datanode that are in the given states. Passing null for one of
+   * of the states acts like a wildcard for that state. This function works by
+   * taking a snapshot of the current collection and then returning the list
+   * from that collection. This means that real map might have changed by the
+   * time we return this list.
+   *
+   * @param opState The operational state of the node
+   * @param health The health of the node
+   * @return List of Datanodes that are known to SCM in the requested states.
+   */
+  @Override
+  public List<DatanodeDetails> getNodes(
+      NodeOperationalState opState, NodeState health) {
+    return nodeStateManager.getNodes(opState, health)
+        .stream()
+        .map(node -> (DatanodeDetails)node).collect(Collectors.toList());
   }
 
   /**
@@ -183,24 +206,60 @@
    * @return count
    */
   @Override
-  public int getNodeCount(NodeState nodestate) {
-    return nodeStateManager.getNodeCount(nodestate);
+  public int getNodeCount(NodeStatus nodeStatus) {
+    return nodeStateManager.getNodeCount(nodeStatus);
   }
 
   /**
-   * Returns the node state of a specific node.
+   * Returns the Number of Datanodes by State they are in. Passing null for
+   * either of the states acts like a wildcard for that state.
    *
-   * @param datanodeDetails Datanode Details
-   * @return Healthy/Stale/Dead/Unknown.
+   * @parem nodeOpState - The Operational State of the node
+   * @param health - The health of the node
+   * @return count
    */
   @Override
-  public NodeState getNodeState(DatanodeDetails datanodeDetails) {
-    try {
-      return nodeStateManager.getNodeState(datanodeDetails);
-    } catch (NodeNotFoundException e) {
-      // TODO: should we throw NodeNotFoundException?
-      return null;
-    }
+  public int getNodeCount(NodeOperationalState nodeOpState, NodeState health) {
+    return nodeStateManager.getNodeCount(nodeOpState, health);
+  }
+
+  /**
+   * Returns the node status of a specific node.
+   *
+   * @param datanodeDetails Datanode Details
+   * @return NodeStatus for the node
+   */
+  @Override
+  public NodeStatus getNodeStatus(DatanodeDetails datanodeDetails)
+      throws NodeNotFoundException {
+    return nodeStateManager.getNodeStatus(datanodeDetails);
+  }
+
+  /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  @Override
+  public void setNodeOperationalState(DatanodeDetails datanodeDetails,
+      NodeOperationalState newState) throws NodeNotFoundException{
+    setNodeOperationalState(datanodeDetails, newState, 0);
+  }
+
+  /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   * @param opStateExpiryEpocSec Seconds from the epoch when the operational
+   *                             state should end. Zero indicates the state
+   *                             never end.
+   */
+  @Override
+  public void setNodeOperationalState(DatanodeDetails datanodeDetails,
+      NodeOperationalState newState, long opStateExpiryEpocSec)
+      throws NodeNotFoundException{
+    nodeStateManager.setNodeOperationalState(
+        datanodeDetails, newState, opStateExpiryEpocSec);
   }
 
   /**
@@ -331,6 +390,7 @@
     try {
       nodeStateManager.updateLastHeartbeatTime(datanodeDetails);
       metrics.incNumHBProcessed();
+      updateDatanodeOpState(datanodeDetails);
     } catch (NodeNotFoundException e) {
       metrics.incNumHBProcessingFailed();
       LOG.error("SCM trying to process heartbeat from an " +
@@ -339,6 +399,41 @@
     return commandQueue.getCommand(datanodeDetails.getUuid());
   }
 
+  /**
+   * If the operational state or expiry reported in the datanode heartbeat do
+   * not match those store in SCM, queue a command to update the state persisted
+   * on the datanode. Additionally, ensure the datanodeDetails stored in SCM
+   * match those reported in the heartbeat.
+   * This method should only be called when processing the
+   * heartbeat, and for a registered node, the information stored in SCM is the
+   * source of truth.
+   * @param reportedDn The DatanodeDetails taken from the node heartbeat.
+   * @throws NodeNotFoundException
+   */
+  private void updateDatanodeOpState(DatanodeDetails reportedDn)
+      throws NodeNotFoundException {
+    NodeStatus scmStatus = getNodeStatus(reportedDn);
+    if (scmStatus.getOperationalState() != reportedDn.getPersistedOpState()
+        || scmStatus.getOpStateExpiryEpochSeconds()
+        != reportedDn.getPersistedOpStateExpiryEpochSec()) {
+      LOG.info("Scheduling a command to update the operationalState " +
+          "persisted on the datanode as the reported value ({}, {}) does not " +
+          "match the value stored in SCM ({}, {})",
+          reportedDn.getPersistedOpState(),
+          reportedDn.getPersistedOpStateExpiryEpochSec(),
+          scmStatus.getOperationalState(),
+          scmStatus.getOpStateExpiryEpochSeconds());
+      commandQueue.addCommand(reportedDn.getUuid(),
+          new SetNodeOperationalStateCommand(
+              Time.monotonicNow(), scmStatus.getOperationalState(),
+              scmStatus.getOpStateExpiryEpochSeconds()));
+    }
+    DatanodeDetails scmDnd = nodeStateManager.getNode(reportedDn);
+    scmDnd.setPersistedOpStateExpiryEpochSec(
+        reportedDn.getPersistedOpStateExpiryEpochSec());
+    scmDnd.setPersistedOpState(reportedDn.getPersistedOpState());
+  }
+
   @Override
   public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) {
     try {
@@ -411,10 +506,10 @@
 
     final Map<DatanodeDetails, SCMNodeStat> nodeStats = new HashMap<>();
 
-    final List<DatanodeInfo> healthyNodes = nodeStateManager
-        .getNodes(NodeState.HEALTHY);
+    final List<DatanodeInfo> healthyNodes =  nodeStateManager
+        .getHealthyNodes();
     final List<DatanodeInfo> staleNodes = nodeStateManager
-        .getNodes(NodeState.STALE);
+        .getStaleNodes();
     final List<DatanodeInfo> datanodes = new ArrayList<>(healthyNodes);
     datanodes.addAll(staleNodes);
 
@@ -463,63 +558,99 @@
     }
   }
 
-  @Override
-  public Map<String, Integer> getNodeCount() {
-    Map<String, Integer> nodeCountMap = new HashMap<String, Integer>();
-    for (NodeState state : NodeState.values()) {
-      nodeCountMap.put(state.toString(), getNodeCount(state));
+  @Override // NodeManagerMXBean
+  public Map<String, Map<String, Integer>> getNodeCount() {
+    Map<String, Map<String, Integer>> nodes = new HashMap<>();
+    for (NodeOperationalState opState : NodeOperationalState.values()) {
+      Map<String, Integer> states = new HashMap<>();
+      for (NodeState health : NodeState.values()) {
+        states.put(health.name(), 0);
+      }
+      nodes.put(opState.name(), states);
     }
-    return nodeCountMap;
+    for (DatanodeInfo dni : nodeStateManager.getAllNodes()) {
+      NodeStatus status = dni.getNodeStatus();
+      nodes.get(status.getOperationalState().name())
+          .compute(status.getHealth().name(), (k, v) -> v+1);
+    }
+    return nodes;
   }
 
   // We should introduce DISK, SSD, etc., notion in
   // SCMNodeStat and try to use it.
-  @Override
+  @Override // NodeManagerMXBean
   public Map<String, Long> getNodeInfo() {
-    long diskCapacity = 0L;
-    long diskUsed = 0L;
-    long diskRemaning = 0L;
-
-    long ssdCapacity = 0L;
-    long ssdUsed = 0L;
-    long ssdRemaining = 0L;
-
-    List<DatanodeInfo> healthyNodes = nodeStateManager
-        .getNodes(NodeState.HEALTHY);
-    List<DatanodeInfo> staleNodes = nodeStateManager
-        .getNodes(NodeState.STALE);
-
-    List<DatanodeInfo> datanodes = new ArrayList<>(healthyNodes);
-    datanodes.addAll(staleNodes);
-
-    for (DatanodeInfo dnInfo : datanodes) {
-      List<StorageReportProto> storageReportProtos = dnInfo.getStorageReports();
-      for (StorageReportProto reportProto : storageReportProtos) {
-        if (reportProto.getStorageType() ==
-            StorageContainerDatanodeProtocolProtos.StorageTypeProto.DISK) {
-          diskCapacity += reportProto.getCapacity();
-          diskRemaning += reportProto.getRemaining();
-          diskUsed += reportProto.getScmUsed();
-        } else if (reportProto.getStorageType() ==
-            StorageContainerDatanodeProtocolProtos.StorageTypeProto.SSD) {
-          ssdCapacity += reportProto.getCapacity();
-          ssdRemaining += reportProto.getRemaining();
-          ssdUsed += reportProto.getScmUsed();
-        }
+    Map<String, Long> nodeInfo = new HashMap<>();
+    // Compute all the possible stats from the enums, and default to zero:
+    for (UsageStates s : UsageStates.values()) {
+      for (UsageMetrics stat : UsageMetrics.values()) {
+        nodeInfo.put(s.label + stat.name(), 0L);
       }
     }
 
-    Map<String, Long> nodeInfo = new HashMap<>();
-    nodeInfo.put("DISKCapacity", diskCapacity);
-    nodeInfo.put("DISKUsed", diskUsed);
-    nodeInfo.put("DISKRemaining", diskRemaning);
-
-    nodeInfo.put("SSDCapacity", ssdCapacity);
-    nodeInfo.put("SSDUsed", ssdUsed);
-    nodeInfo.put("SSDRemaining", ssdRemaining);
+    for (DatanodeInfo node : nodeStateManager.getAllNodes()) {
+      String keyPrefix = "";
+      NodeStatus status = node.getNodeStatus();
+      if (status.isMaintenance()) {
+        keyPrefix = UsageStates.MAINT.getLabel();
+      } else if (status.isDecommission()) {
+        keyPrefix = UsageStates.DECOM.getLabel();
+      } else if (status.isAlive()) {
+        // Inservice but not dead
+        keyPrefix = UsageStates.ONLINE.getLabel();
+      } else {
+        // dead inservice node, skip it
+        continue;
+      }
+      List<StorageReportProto> storageReportProtos = node.getStorageReports();
+      for (StorageReportProto reportProto : storageReportProtos) {
+        if (reportProto.getStorageType() ==
+            StorageContainerDatanodeProtocolProtos.StorageTypeProto.DISK) {
+          nodeInfo.compute(keyPrefix + UsageMetrics.DiskCapacity.name(),
+              (k, v) -> v + reportProto.getCapacity());
+          nodeInfo.compute(keyPrefix + UsageMetrics.DiskRemaining.name(),
+              (k, v) -> v + reportProto.getRemaining());
+          nodeInfo.compute(keyPrefix + UsageMetrics.DiskUsed.name(),
+              (k, v) -> v + reportProto.getScmUsed());
+        } else if (reportProto.getStorageType() ==
+            StorageContainerDatanodeProtocolProtos.StorageTypeProto.SSD) {
+          nodeInfo.compute(keyPrefix + UsageMetrics.SSDCapacity.name(),
+              (k, v) -> v + reportProto.getCapacity());
+          nodeInfo.compute(keyPrefix + UsageMetrics.SSDRemaining.name(),
+              (k, v) -> v + reportProto.getRemaining());
+          nodeInfo.compute(keyPrefix + UsageMetrics.SSDUsed.name(),
+              (k, v) -> v + reportProto.getScmUsed());
+        }
+      }
+    }
     return nodeInfo;
   }
 
+  private enum UsageMetrics {
+    DiskCapacity,
+    DiskUsed,
+    DiskRemaining,
+    SSDCapacity,
+    SSDUsed,
+    SSDRemaining
+  }
+
+  private enum UsageStates {
+    ONLINE(""),
+    MAINT("Maintenance"),
+    DECOM("Decommissioned");
+
+    private final String label;
+
+    public String getLabel() {
+      return label;
+    }
+
+    UsageStates(String label) {
+      this.label = label;
+    }
+  }
+
   /**
    * Returns the min of no healthy volumes reported out of the set
    * of datanodes constituting the pipeline.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java
index c515f2f..5ea4d91 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeMetrics.java
@@ -23,6 +23,7 @@
 import org.apache.hadoop.hdds.annotation.InterfaceAudience;
 import org.apache.hadoop.metrics2.MetricsCollector;
 import org.apache.hadoop.metrics2.MetricsInfo;
+import org.apache.hadoop.metrics2.MetricsRecordBuilder;
 import org.apache.hadoop.metrics2.MetricsSource;
 import org.apache.hadoop.metrics2.MetricsSystem;
 import org.apache.hadoop.metrics2.annotation.Metric;
@@ -32,12 +33,7 @@
 import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.ozone.OzoneConsts;
-
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DECOMMISSIONED;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DECOMMISSIONING;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
+import org.apache.hadoop.util.StringUtils;
 
 /**
  * This class maintains Node related metrics.
@@ -53,6 +49,7 @@
   private @Metric MutableCounterLong numHBProcessingFailed;
   private @Metric MutableCounterLong numNodeReportProcessed;
   private @Metric MutableCounterLong numNodeReportProcessingFailed;
+  private @Metric String textMetric;
 
   private final MetricsRegistry registry;
   private final NodeManagerMXBean managerMXBean;
@@ -63,6 +60,7 @@
   private SCMNodeMetrics(NodeManagerMXBean managerMXBean) {
     this.managerMXBean = managerMXBean;
     this.registry = new MetricsRegistry(recordInfo);
+    this.textMetric = "my_test_metric";
   }
 
   /**
@@ -118,44 +116,58 @@
   @Override
   @SuppressWarnings("SuspiciousMethodCalls")
   public void getMetrics(MetricsCollector collector, boolean all) {
-    Map<String, Integer> nodeCount = managerMXBean.getNodeCount();
+    Map<String, Map<String, Integer>> nodeCount = managerMXBean.getNodeCount();
     Map<String, Long> nodeInfo = managerMXBean.getNodeInfo();
-    registry.snapshot(
-        collector.addRecord(registry.info()) // Add annotated ones first
-            .addGauge(Interns.info(
-                "HealthyNodes",
-                "Number of healthy datanodes"),
-                nodeCount.get(HEALTHY.toString()))
-            .addGauge(Interns.info("StaleNodes",
-                "Number of stale datanodes"),
-                nodeCount.get(STALE.toString()))
-            .addGauge(Interns.info("DeadNodes",
-                "Number of dead datanodes"),
-                nodeCount.get(DEAD.toString()))
-            .addGauge(Interns.info("DecommissioningNodes",
-                "Number of decommissioning datanodes"),
-                nodeCount.get(DECOMMISSIONING.toString()))
-            .addGauge(Interns.info("DecommissionedNodes",
-                "Number of decommissioned datanodes"),
-                nodeCount.get(DECOMMISSIONED.toString()))
-            .addGauge(Interns.info("DiskCapacity",
-                "Total disk capacity"),
-                nodeInfo.get("DISKCapacity"))
-            .addGauge(Interns.info("DiskUsed",
-                "Total disk capacity used"),
-                nodeInfo.get("DISKUsed"))
-            .addGauge(Interns.info("DiskRemaining",
-                "Total disk capacity remaining"),
-                nodeInfo.get("DISKRemaining"))
-            .addGauge(Interns.info("SSDCapacity",
-                "Total ssd capacity"),
-                nodeInfo.get("SSDCapacity"))
-            .addGauge(Interns.info("SSDUsed",
-                "Total ssd capacity used"),
-                nodeInfo.get("SSDUsed"))
-            .addGauge(Interns.info("SSDRemaining",
-                "Total disk capacity remaining"),
-                nodeInfo.get("SSDRemaining")),
-        all);
+
+    /**
+     * Loop over the Node map and create a metric for the cross product of all
+     * Operational and health states, ie:
+     *     InServiceHealthy
+     *     InServiceStale
+     *     ...
+     *     EnteringMaintenanceHealthy
+     *     ...
+     */
+    MetricsRecordBuilder metrics = collector.addRecord(registry.info());
+    for(Map.Entry<String, Map<String, Integer>> e : nodeCount.entrySet()) {
+      for(Map.Entry<String, Integer> h : e.getValue().entrySet()) {
+        metrics.addGauge(
+            Interns.info(
+                StringUtils.camelize(e.getKey()+"_"+h.getKey()+"_nodes"),
+                "Number of "+e.getKey()+" "+h.getKey()+" datanodes"),
+            h.getValue());
+      }
+    }
+
+    for (Map.Entry<String, Long> e : nodeInfo.entrySet()) {
+      metrics.addGauge(
+          Interns.info(e.getKey(), diskMetricDescription(e.getKey())),
+          e.getValue());
+    }
+    registry.snapshot(metrics, all);
+  }
+
+  private String diskMetricDescription(String metric) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Total");
+    if (metric.indexOf("Maintenance") >= 0) {
+      sb.append(" maintenance");
+    } else if (metric.indexOf("Decommissioned") >= 0) {
+      sb.append(" decommissioned");
+    }
+    if (metric.indexOf("DiskCapacity") >= 0) {
+      sb.append(" disk capacity");
+    } else if (metric.indexOf("DiskUsed") >= 0) {
+      sb.append(" disk capacity used");
+    } else if (metric.indexOf("DiskRemaining") >= 0) {
+      sb.append(" disk capacity remaining");
+    } else if (metric.indexOf("SSDCapacity") >= 0) {
+      sb.append(" SSD capacity");
+    } else if (metric.indexOf("SSDUsed") >= 0) {
+      sb.append(" SSD capacity used");
+    } else if (metric.indexOf("SSDRemaining") >= 0) {
+      sb.append(" SSD capacity remaining");
+    }
+    return sb.toString();
   }
 }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java
new file mode 100644
index 0000000..9418a7a
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/StartDatanodeAdminHandler.java
@@ -0,0 +1,68 @@
+/**
+ * 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.node;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.server.events.EventHandler;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Set;
+
+/**
+ * Handler which is fired when a datanode starts admin (decommission or
+ * maintenance).
+ */
+public class StartDatanodeAdminHandler
+    implements EventHandler<DatanodeDetails> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(StartDatanodeAdminHandler.class);
+
+  private final NodeManager nodeManager;
+  private final PipelineManager pipelineManager;
+
+  public StartDatanodeAdminHandler(NodeManager nodeManager,
+      PipelineManager pipelineManager) {
+    this.nodeManager = nodeManager;
+    this.pipelineManager = pipelineManager;
+  }
+
+  @Override
+  public void onMessage(DatanodeDetails datanodeDetails,
+                        EventPublisher publisher) {
+    Set<PipelineID> pipelineIds =
+        nodeManager.getPipelines(datanodeDetails);
+    LOG.info("Admin start on datanode {}. Finalizing its pipelines {}",
+        datanodeDetails, pipelineIds);
+    for (PipelineID pipelineID : pipelineIds) {
+      try {
+        Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
+        pipelineManager.finalizeAndDestroyPipeline(pipeline, false);
+      } catch (IOException e) {
+        LOG.info("Could not finalize pipeline={} for dn={}", pipelineID,
+            datanodeDetails);
+      }
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java
index baebef5..d44f6b7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/states/NodeStateMap.java
@@ -20,13 +20,16 @@
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.stream.Collectors;
 
 /**
  * Maintains the state of datanodes in SCM. This class should only be used by
@@ -35,16 +38,11 @@
  * this class.
  */
 public class NodeStateMap {
-
   /**
    * Node id to node info map.
    */
   private final ConcurrentHashMap<UUID, DatanodeInfo> nodeMap;
   /**
-   * Represents the current state of node.
-   */
-  private final ConcurrentHashMap<NodeState, Set<UUID>> stateMap;
-  /**
    * Node to set of containers on the node.
    */
   private final ConcurrentHashMap<UUID, Set<ContainerID>> nodeToContainer;
@@ -57,29 +55,18 @@
   public NodeStateMap() {
     lock = new ReentrantReadWriteLock();
     nodeMap = new ConcurrentHashMap<>();
-    stateMap = new ConcurrentHashMap<>();
     nodeToContainer = new ConcurrentHashMap<>();
-    initStateMap();
-  }
-
-  /**
-   * Initializes the state map with available states.
-   */
-  private void initStateMap() {
-    for (NodeState state : NodeState.values()) {
-      stateMap.put(state, ConcurrentHashMap.newKeySet());
-    }
   }
 
   /**
    * Adds a node to NodeStateMap.
    *
    * @param datanodeDetails DatanodeDetails
-   * @param nodeState initial NodeState
+   * @param nodeStatus initial NodeStatus
    *
    * @throws NodeAlreadyExistsException if the node already exist
    */
-  public void addNode(DatanodeDetails datanodeDetails, NodeState nodeState)
+  public void addNode(DatanodeDetails datanodeDetails, NodeStatus nodeStatus)
       throws NodeAlreadyExistsException {
     lock.writeLock().lock();
     try {
@@ -87,34 +74,55 @@
       if (nodeMap.containsKey(id)) {
         throw new NodeAlreadyExistsException("Node UUID: " + id);
       }
-      nodeMap.put(id, new DatanodeInfo(datanodeDetails));
-      nodeToContainer.put(id, ConcurrentHashMap.newKeySet());
-      stateMap.get(nodeState).add(id);
+      nodeMap.put(id, new DatanodeInfo(datanodeDetails, nodeStatus));
+      nodeToContainer.put(id, new HashSet<>());
     } finally {
       lock.writeLock().unlock();
     }
   }
 
   /**
-   * Updates the node state.
+   * Updates the node health state.
    *
    * @param nodeId Node Id
-   * @param currentState current state
-   * @param newState new state
+   * @param newHealth new health state
    *
    * @throws NodeNotFoundException if the node is not present
    */
-  public void updateNodeState(UUID nodeId, NodeState currentState,
-                              NodeState newState)throws NodeNotFoundException {
-    lock.writeLock().lock();
+  public NodeStatus updateNodeHealthState(UUID nodeId, NodeState newHealth)
+      throws NodeNotFoundException {
     try {
-      checkIfNodeExist(nodeId);
-      if (stateMap.get(currentState).remove(nodeId)) {
-        stateMap.get(newState).add(nodeId);
-      } else {
-        throw new NodeNotFoundException("Node UUID: " + nodeId +
-            ", not found in state: " + currentState);
-      }
+      lock.writeLock().lock();
+      DatanodeInfo dn = getNodeInfo(nodeId);
+      NodeStatus oldStatus = dn.getNodeStatus();
+      NodeStatus newStatus = new NodeStatus(
+          oldStatus.getOperationalState(), newHealth);
+      dn.setNodeStatus(newStatus);
+      return newStatus;
+    } finally {
+      lock.writeLock().unlock();
+    }
+  }
+
+  /**
+   * Updates the node operational state.
+   *
+   * @param nodeId Node Id
+   * @param newOpState new operational state
+   *
+   * @throws NodeNotFoundException if the node is not present
+   */
+  public NodeStatus updateNodeOperationalState(UUID nodeId,
+      NodeOperationalState newOpState, long opStateExpiryEpochSeconds)
+      throws NodeNotFoundException {
+    try {
+      lock.writeLock().lock();
+      DatanodeInfo dn = getNodeInfo(nodeId);
+      NodeStatus oldStatus = dn.getNodeStatus();
+      NodeStatus newStatus = new NodeStatus(
+          newOpState, oldStatus.getHealth(), opStateExpiryEpochSeconds);
+      dn.setNodeStatus(newStatus);
+      return newStatus;
     } finally {
       lock.writeLock().unlock();
     }
@@ -139,21 +147,38 @@
     }
   }
 
-
   /**
    * Returns the list of node ids which are in the specified state.
    *
-   * @param state NodeState
+   * @param status NodeStatus
    *
    * @return list of node ids
    */
-  public List<UUID> getNodes(NodeState state) {
-    lock.readLock().lock();
-    try {
-      return new ArrayList<>(stateMap.get(state));
-    } finally {
-      lock.readLock().unlock();
+  public List<UUID> getNodes(NodeStatus status) {
+    ArrayList<UUID> nodes = new ArrayList<>();
+    for (DatanodeInfo dn : filterNodes(status)) {
+      nodes.add(dn.getUuid());
     }
+    return nodes;
+  }
+
+  /**
+   * Returns the list of node ids which match the desired operational state
+   * and health. Passing a null for either value is equivalent to a wild card.
+   *
+   * Therefore, passing opState = null, health=stale will return all stale nodes
+   * regardless of their operational state.
+   *
+   * @param opState
+   * @param health
+   * @return The list of nodes matching the given states
+   */
+  public List<UUID> getNodes(NodeOperationalState opState, NodeState health) {
+    ArrayList<UUID> nodes = new ArrayList<>();
+    for (DatanodeInfo dn : filterNodes(opState, health)) {
+      nodes.add(dn.getUuid());
+    }
+    return nodes;
   }
 
   /**
@@ -162,8 +187,8 @@
    * @return list of all the node ids
    */
   public List<UUID> getAllNodes() {
-    lock.readLock().lock();
     try {
+      lock.readLock().lock();
       return new ArrayList<>(nodeMap.keySet());
     } finally {
       lock.readLock().unlock();
@@ -171,22 +196,72 @@
   }
 
   /**
-   * Returns the count of nodes in the specified state.
+   * Returns the list of all the nodes as DatanodeInfo objects.
    *
-   * @param state NodeState
-   *
-   * @return Number of nodes in the specified state
+   * @return list of all the node ids
    */
-  public int getNodeCount(NodeState state) {
-    lock.readLock().lock();
+  public List<DatanodeInfo> getAllDatanodeInfos() {
     try {
-      return stateMap.get(state).size();
+      lock.readLock().lock();
+      return new ArrayList<>(nodeMap.values());
     } finally {
       lock.readLock().unlock();
     }
   }
 
   /**
+   * Returns a list of the nodes as DatanodeInfo objects matching the passed
+   * status.
+   *
+   * @param status - The status of the nodes to return
+   * @return List of DatanodeInfo for the matching nodes
+   */
+  public List<DatanodeInfo> getDatanodeInfos(NodeStatus status) {
+    return filterNodes(status);
+  }
+
+  /**
+   * Returns a list of the nodes as DatanodeInfo objects matching the passed
+   * states. Passing null for either of the state values acts as a wildcard
+   * for that state.
+   *
+   * @param opState - The node operational state
+   * @param health - The node health
+   * @return List of DatanodeInfo for the matching nodes
+   */
+  public List<DatanodeInfo> getDatanodeInfos(
+      NodeOperationalState opState, NodeState health) {
+    return filterNodes(opState, health);
+  }
+
+  /**
+   * Returns the count of nodes in the specified state.
+   *
+   * @param state NodeStatus
+   *
+   * @return Number of nodes in the specified state
+   */
+  public int getNodeCount(NodeStatus state) {
+    return getNodes(state).size();
+  }
+
+  /**
+   * Returns the count of node ids which match the desired operational state
+   * and health. Passing a null for either value is equivalent to a wild card.
+   *
+   * Therefore, passing opState=null, health=stale will count all stale nodes
+   * regardless of their operational state.
+   *
+   * @param opState
+   * @param health
+   *
+   * @return Number of nodes in the specified state
+   */
+  public int getNodeCount(NodeOperationalState opState, NodeState health) {
+    return getNodes(opState, health).size();
+  }
+
+  /**
    * Returns the total node count.
    *
    * @return node count
@@ -209,17 +284,15 @@
    *
    * @throws NodeNotFoundException if the node is not found
    */
-  public NodeState getNodeState(UUID uuid) throws NodeNotFoundException {
+  public NodeStatus getNodeStatus(UUID uuid) throws NodeNotFoundException {
     lock.readLock().lock();
     try {
-      checkIfNodeExist(uuid);
-      for (Map.Entry<NodeState, Set<UUID>> entry : stateMap.entrySet()) {
-        if (entry.getValue().contains(uuid)) {
-          return entry.getKey();
-        }
+      DatanodeInfo dn = nodeMap.get(uuid);
+      if (dn == null) {
+        throw new NodeNotFoundException("Node not found in node map." +
+            " UUID: " + uuid);
       }
-      throw new NodeNotFoundException("Node not found in node state map." +
-          " UUID: " + uuid);
+      return dn.getNodeStatus();
     } finally {
       lock.readLock().unlock();
     }
@@ -289,12 +362,13 @@
    */
   @Override
   public String toString() {
+    // TODO - fix this method to include the commented out values
     StringBuilder builder = new StringBuilder();
     builder.append("Total number of nodes: ").append(getTotalNodeCount());
-    for (NodeState state : NodeState.values()) {
-      builder.append("Number of nodes in ").append(state).append(" state: ")
-          .append(getNodeCount(state));
-    }
+   // for (NodeState state : NodeState.values()) {
+   //   builder.append("Number of nodes in ").append(state).append(" state: ")
+   //       .append(getNodeCount(state));
+   // }
     return builder.toString();
   }
 
@@ -309,4 +383,50 @@
       throw new NodeNotFoundException("Node UUID: " + uuid);
     }
   }
+
+  /**
+   * Create a list of datanodeInfo for all nodes matching the passed states.
+   * Passing null for one of the states acts like a wildcard for that state.
+   *
+   * @param opState
+   * @param health
+   * @return List of DatanodeInfo objects matching the passed state
+   */
+  private List<DatanodeInfo> filterNodes(
+      NodeOperationalState opState, NodeState health) {
+    if (opState != null && health != null) {
+      return filterNodes(new NodeStatus(opState, health));
+    }
+    if (opState == null && health == null) {
+      return getAllDatanodeInfos();
+    }
+    try {
+      lock.readLock().lock();
+      return nodeMap.values().stream()
+          .filter(n -> opState == null
+              || n.getNodeStatus().getOperationalState() == opState)
+          .filter(n -> health == null
+              || n.getNodeStatus().getHealth() == health)
+          .collect(Collectors.toList());
+    } finally {
+      lock.readLock().unlock();
+    }
+  }
+
+  /**
+   * Create a list of datanodeInfo for all nodes matching the passsed status.
+   *
+   * @param status
+   * @return List of DatanodeInfo objects matching the passed state
+   */
+  private List<DatanodeInfo> filterNodes(NodeStatus status) {
+    try {
+      lock.readLock().lock();
+      return nodeMap.values().stream()
+          .filter(n -> n.getNodeStatus().equals(status))
+          .collect(Collectors.toList());
+    }  finally {
+      lock.readLock().unlock();
+    }
+  }
 }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java
index b9441be..233097d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelinePlacementPolicy.java
@@ -28,6 +28,7 @@
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -123,7 +124,7 @@
       throws SCMException {
     // get nodes in HEALTHY state
     List<DatanodeDetails> healthyNodes =
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY);
+        nodeManager.getNodes(NodeStatus.inServiceHealthy());
     boolean multipleRacks = multipleRacksAvailable(healthyNodes);
     if (excludedNodes != null) {
       healthyNodes.removeAll(excludedNodes);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
index 533f77e..8df976c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineProvider.java
@@ -25,11 +25,11 @@
 import java.util.stream.Collectors;
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 
 /**
  * Interface for creating pipelines.
@@ -79,7 +79,7 @@
 
     // Get list of healthy nodes
     List<DatanodeDetails> dns = nodeManager
-        .getNodes(HddsProtos.NodeState.HEALTHY)
+        .getNodes(NodeStatus.inServiceHealthy())
         .parallelStream()
         .filter(dn -> !dnsUsed.contains(dn))
         .limit(factor.getNumber())
@@ -89,7 +89,7 @@
           .format("Cannot create pipeline of factor %d using %d nodes." +
                   " Used %d nodes. Healthy nodes %d", factor.getNumber(),
               dns.size(), dnsUsed.size(),
-              nodeManager.getNodes(HddsProtos.NodeState.HEALTHY).size());
+              nodeManager.getNodes(NodeStatus.inServiceHealthy()).size());
       throw new SCMException(e,
           SCMException.ResultCodes.FAILED_TO_FIND_SUITABLE_NODE);
     }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
index 830db18..cd468bc 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/RatisPipelineProvider.java
@@ -23,13 +23,13 @@
 
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
 import org.apache.hadoop.hdds.scm.pipeline.leader.choose.algorithms.LeaderChoosePolicy;
 import org.apache.hadoop.hdds.scm.pipeline.leader.choose.algorithms.LeaderChoosePolicyFactory;
@@ -91,7 +91,7 @@
           ReplicationType.RATIS, factor).size() -
           getPipelineStateManager().getPipelines(ReplicationType.RATIS, factor,
               PipelineState.CLOSED).size()) > maxPipelinePerDatanode *
-          getNodeManager().getNodeCount(HddsProtos.NodeState.HEALTHY) /
+          getNodeManager().getNodeCount(NodeStatus.inServiceHealthy()) /
           factor.getNumber();
     }
 
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
index c7b6305..f1e6c1b 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SimplePipelineProvider.java
@@ -42,7 +42,6 @@
   public Pipeline create(ReplicationFactor factor) throws IOException {
     List<DatanodeDetails> dns = pickNodesNeverUsed(ReplicationType.STAND_ALONE,
         factor);
-
     if (dns.size() < factor.getNumber()) {
       String e = String
           .format("Cannot create pipeline of factor %d using %d nodes.",
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
index d5496b4..f4ee6d1 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java
@@ -65,6 +65,12 @@
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartReplicationManagerResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopReplicationManagerResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionNodesRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionNodesResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.RecommissionNodesRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.RecommissionNodesResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartMaintenanceNodesRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartMaintenanceNodesResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetSafeModeRuleStatusesRequestProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetSafeModeRuleStatusesResponseProto;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.SafeModeRuleStatusProto;
@@ -128,6 +134,7 @@
             request.getTraceID());
   }
 
+  @SuppressWarnings("checkstyle:methodlength")
   public ScmContainerLocationResponse processRequest(
       ScmContainerLocationRequest request) throws ServiceException {
     try {
@@ -270,6 +277,27 @@
             .setGetSafeModeRuleStatusesResponse(getSafeModeRuleStatues(
                 request.getGetSafeModeRuleStatusesRequest()))
             .build();
+      case DecommissionNodes:
+        return ScmContainerLocationResponse.newBuilder()
+            .setCmdType(request.getCmdType())
+            .setStatus(Status.OK)
+            .setDecommissionNodesResponse(decommissionNodes(
+                request.getDecommissionNodesRequest()))
+            .build();
+      case RecommissionNodes:
+        return ScmContainerLocationResponse.newBuilder()
+            .setCmdType(request.getCmdType())
+            .setStatus(Status.OK)
+            .setRecommissionNodesResponse(recommissionNodes(
+                request.getRecommissionNodesRequest()))
+            .build();
+      case StartMaintenanceNodes:
+        return ScmContainerLocationResponse.newBuilder()
+            .setCmdType(request.getCmdType())
+            .setStatus(Status.OK)
+            .setStartMaintenanceNodesResponse(startMaintenanceNodes(
+                request.getStartMaintenanceNodesRequest()))
+          .build();
       default:
         throw new IllegalArgumentException(
             "Unknown command type: " + request.getCmdType());
@@ -356,13 +384,19 @@
       StorageContainerLocationProtocolProtos.NodeQueryRequestProto request)
       throws IOException {
 
-    HddsProtos.NodeState nodeState = request.getState();
-    List<HddsProtos.Node> datanodes = impl.queryNode(nodeState,
+    HddsProtos.NodeOperationalState opState = null;
+    HddsProtos.NodeState nodeState = null;
+    if (request.hasState()) {
+      nodeState = request.getState();
+    }
+    if (request.hasOpState()) {
+      opState = request.getOpState();
+    }
+    List<HddsProtos.Node> datanodes = impl.queryNode(opState, nodeState,
         request.getScope(), request.getPoolName());
     return NodeQueryResponseProto.newBuilder()
         .addAllDatanodes(datanodes)
         .build();
-
   }
 
   public SCMCloseContainerResponseProto closeContainer(
@@ -497,4 +531,25 @@
         .setIsRunning(impl.getReplicationManagerStatus()).build();
   }
 
+  public DecommissionNodesResponseProto decommissionNodes(
+      DecommissionNodesRequestProto request) throws IOException {
+    impl.decommissionNodes(request.getHostsList());
+    return DecommissionNodesResponseProto.newBuilder()
+        .build();
+  }
+
+  public RecommissionNodesResponseProto recommissionNodes(
+      RecommissionNodesRequestProto request) throws IOException {
+    impl.recommissionNodes(request.getHostsList());
+    return RecommissionNodesResponseProto.newBuilder().build();
+  }
+
+  public StartMaintenanceNodesResponseProto startMaintenanceNodes(
+      StartMaintenanceNodesRequestProto request) throws IOException {
+    impl.startMaintenanceNodes(request.getHostsList(),
+        (int)request.getEndInHours());
+    return StartMaintenanceNodesResponseProto.newBuilder()
+        .build();
+  }
+
 }
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index df5c147..171d267 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -21,12 +21,17 @@
  */
 package org.apache.hadoop.hdds.scm.server;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Maps;
+import com.google.protobuf.BlockingService;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
 import java.util.stream.Collectors;
 
 import org.apache.commons.lang3.tuple.Pair;
@@ -38,23 +43,25 @@
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos;
 import org.apache.hadoop.hdds.scm.ScmInfo;
 import org.apache.hadoop.hdds.scm.ScmUtils;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
+import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.ContainerInfo;
 import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
 import org.apache.hadoop.hdds.scm.container.ContainerReplica;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
-import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
-import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
 import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocolServerSideTranslatorPB;
 import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
 import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus;
-import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.hdds.utils.HddsServerUtil;
@@ -71,10 +78,6 @@
 import org.apache.hadoop.ozone.audit.Auditor;
 import org.apache.hadoop.ozone.audit.SCMAction;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Maps;
-import com.google.protobuf.BlockingService;
 import com.google.protobuf.ProtocolMessageEnum;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService.newReflectiveBlockingService;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
@@ -381,7 +384,8 @@
   }
 
   @Override
-  public List<HddsProtos.Node> queryNode(HddsProtos.NodeState state,
+  public List<HddsProtos.Node> queryNode(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState state,
       HddsProtos.QueryScope queryScope, String poolName) throws
       IOException {
 
@@ -390,13 +394,57 @@
     }
 
     List<HddsProtos.Node> result = new ArrayList<>();
-    queryNode(state).forEach(node -> result.add(HddsProtos.Node.newBuilder()
-        .setNodeID(node.getProtoBufMessage())
-        .addNodeStates(state)
-        .build()));
-
+    for (DatanodeDetails node : queryNode(opState, state)) {
+      try {
+        NodeStatus ns = scm.getScmNodeManager().getNodeStatus(node);
+        result.add(HddsProtos.Node.newBuilder()
+            .setNodeID(node.getProtoBufMessage())
+            .addNodeStates(ns.getHealth())
+            .addNodeOperationalStates(ns.getOperationalState())
+            .build());
+      } catch (NodeNotFoundException e) {
+        throw new IOException(
+            "An unexpected error occurred querying the NodeStatus", e);
+      }
+    }
     return result;
+  }
 
+  @Override
+  public void decommissionNodes(List<String> nodes) throws IOException {
+    String remoteUser = getRpcRemoteUsername();
+    try {
+      getScm().checkAdminAccess(remoteUser);
+      scm.getScmDecommissionManager().decommissionNodes(nodes);
+    } catch (Exception ex) {
+      LOG.error("Failed to decommission nodes", ex);
+      throw ex;
+    }
+  }
+
+  @Override
+  public void recommissionNodes(List<String> nodes) throws IOException {
+    String remoteUser = getRpcRemoteUsername();
+    try {
+      getScm().checkAdminAccess(remoteUser);
+      scm.getScmDecommissionManager().recommissionNodes(nodes);
+    } catch (Exception ex) {
+      LOG.error("Failed to recommission nodes", ex);
+      throw ex;
+    }
+  }
+
+  @Override
+  public void startMaintenanceNodes(List<String> nodes, int endInHours)
+      throws IOException {
+    String remoteUser = getRpcRemoteUsername();
+    try {
+      getScm().checkAdminAccess(remoteUser);
+      scm.getScmDecommissionManager().startMaintenanceNodes(nodes, endInHours);
+    } catch (Exception ex) {
+      LOG.error("Failed to place nodes into maintenance mode", ex);
+      throw ex;
+    }
   }
 
   @Override
@@ -571,12 +619,13 @@
    * operation between the
    * operators.
    *
-   * @param state - NodeStates.
+   * @param opState - NodeOperational State
+   * @param state - NodeState.
    * @return List of Datanodes.
    */
-  public List<DatanodeDetails> queryNode(HddsProtos.NodeState state) {
-    Preconditions.checkNotNull(state, "Node Query set cannot be null");
-    return queryNodeState(state);
+  public List<DatanodeDetails> queryNode(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState state) {
+    return new ArrayList<>(queryNodeState(opState, state));
   }
 
   @VisibleForTesting
@@ -595,11 +644,19 @@
   /**
    * Query the System for Nodes.
    *
+   * @params opState - The node operational state
    * @param nodeState - NodeState that we are interested in matching.
-   * @return List of Datanodes that match the NodeState.
+   * @return Set of Datanodes that match the NodeState.
    */
-  private List<DatanodeDetails> queryNodeState(HddsProtos.NodeState nodeState) {
-    return scm.getScmNodeManager().getNodes(nodeState);
+  private Set<DatanodeDetails> queryNodeState(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState nodeState) {
+    Set<DatanodeDetails> returnSet = new TreeSet<>();
+    List<DatanodeDetails> tmp = scm.getScmNodeManager()
+        .getNodes(opState, nodeState);
+    if ((tmp != null) && (tmp.size() > 0)) {
+      returnSet.addAll(tmp);
+    }
+    return returnSet;
   }
 
   @Override
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
index a295341..c2dafcb 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMDatanodeProtocolServer.java
@@ -69,6 +69,7 @@
 import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
 import org.apache.hadoop.ozone.protocol.commands.ReplicateContainerCommand;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolPB;
 import org.apache.hadoop.ozone.protocolPB.StorageContainerDatanodeProtocolServerSideTranslatorPB;
 import org.apache.hadoop.security.authorize.PolicyProvider;
@@ -85,6 +86,7 @@
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.deleteContainerCommand;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.replicateContainerCommand;
 import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.reregisterCommand;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type.setNodeOperationalStateCommand;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY;
@@ -344,6 +346,12 @@
           .setClosePipelineCommandProto(
               ((ClosePipelineCommand)cmd).getProto())
           .build();
+    case setNodeOperationalStateCommand:
+      return builder
+          .setCommandType(setNodeOperationalStateCommand)
+          .setSetNodeOperationalStateCommandProto(
+              ((SetNodeOperationalStateCommand)cmd).getProto())
+          .build();
     default:
       throw new IllegalArgumentException("Scm command " +
           cmd.getType().toString() + " is not implemented");
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 3cf12e7..2e9f576 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -74,11 +74,13 @@
 import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
 import org.apache.hadoop.hdds.scm.node.DeadNodeHandler;
 import org.apache.hadoop.hdds.scm.node.NewNodeHandler;
+import org.apache.hadoop.hdds.scm.node.StartDatanodeAdminHandler;
+import org.apache.hadoop.hdds.scm.node.NonHealthyToHealthyNodeHandler;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.hdds.scm.node.NodeReportHandler;
-import org.apache.hadoop.hdds.scm.node.NonHealthyToHealthyNodeHandler;
 import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
 import org.apache.hadoop.hdds.scm.node.StaleNodeHandler;
+import org.apache.hadoop.hdds.scm.node.NodeDecommissionManager;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineActionHandler;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineReportHandler;
@@ -162,6 +164,7 @@
   private ContainerManager containerManager;
   private BlockManager scmBlockManager;
   private final SCMStorageConfig scmStorageConfig;
+  private NodeDecommissionManager scmDecommissionManager;
 
   private SCMMetadataStore scmMetadataStore;
 
@@ -293,11 +296,14 @@
     CommandStatusReportHandler cmdStatusReportHandler =
         new CommandStatusReportHandler();
 
-    NewNodeHandler newNodeHandler = new NewNodeHandler(pipelineManager, conf);
+    NewNodeHandler newNodeHandler = new NewNodeHandler(pipelineManager,
+        scmDecommissionManager, conf);
     StaleNodeHandler staleNodeHandler =
         new StaleNodeHandler(scmNodeManager, pipelineManager, conf);
     DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager,
         pipelineManager, containerManager);
+    StartDatanodeAdminHandler datanodeStartAdminHandler =
+        new StartDatanodeAdminHandler(scmNodeManager, pipelineManager);
     NonHealthyToHealthyNodeHandler nonHealthyToHealthyNodeHandler =
         new NonHealthyToHealthyNodeHandler(pipelineManager, conf);
     ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
@@ -334,7 +340,6 @@
     blockProtocolServer = new SCMBlockProtocolServer(conf, this);
     clientProtocolServer = new SCMClientProtocolServer(conf, this);
     httpServer = new StorageContainerManagerHttpServer(conf);
-
     eventQueue.addHandler(SCMEvents.DATANODE_COMMAND, scmNodeManager);
     eventQueue.addHandler(SCMEvents.RETRIABLE_DATANODE_COMMAND, scmNodeManager);
     eventQueue.addHandler(SCMEvents.NODE_REPORT, nodeReportHandler);
@@ -348,6 +353,8 @@
     eventQueue.addHandler(SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE,
         nonHealthyToHealthyNodeHandler);
     eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
+    eventQueue.addHandler(SCMEvents.START_ADMIN_ON_NODE,
+        datanodeStartAdminHandler);
     eventQueue.addHandler(SCMEvents.CMD_STATUS_REPORT, cmdStatusReportHandler);
     eventQueue
         .addHandler(SCMEvents.PENDING_DELETE_STATUS, pendingDeleteHandler);
@@ -448,6 +455,8 @@
       scmSafeModeManager = new SCMSafeModeManager(conf,
           containerManager.getContainers(), pipelineManager, eventQueue);
     }
+    scmDecommissionManager = new NodeDecommissionManager(conf, scmNodeManager,
+        containerManager, eventQueue, replicationManager);
   }
 
   /**
@@ -836,6 +845,13 @@
     }
 
     try {
+      LOG.info("Stopping the Datanode Admin Monitor.");
+      scmDecommissionManager.stop();
+    } catch (Exception ex) {
+      LOG.error("The Datanode Admin Monitor failed to stop", ex);
+    }
+
+    try {
       LOG.info("Stopping Lease Manager of the command watchers");
       commandWatcherLeaseManager.shutdown();
     } catch (Exception ex) {
@@ -950,7 +966,18 @@
    * @return int -- count
    */
   public int getNodeCount(NodeState nodestate) {
-    return scmNodeManager.getNodeCount(nodestate);
+    // TODO - decomm - this probably needs to accept opState and health
+    return scmNodeManager.getNodeCount(null, nodestate);
+  }
+
+  /**
+   * Returns the node decommission manager.
+   *
+   * @return NodeDecommissionManager The decommission manger for the used by
+   *         scm
+   */
+  public NodeDecommissionManager getScmDecommissionManager() {
+    return scmDecommissionManager;
   }
 
   /**
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 6b6e8d8..82bdd60 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -34,6 +34,7 @@
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.TestUtils;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
@@ -505,8 +506,8 @@
     nodeManager.setNumHealthyVolumes(1);
     // create pipelines
     for (int i = 0;
-         i < nodeManager.getNodes(HddsProtos.NodeState.HEALTHY).size() / factor
-             .getNumber(); i++) {
+         i < nodeManager.getNodes(NodeStatus.inServiceHealthy()).size()
+             / factor.getNumber(); i++) {
       pipelineManager.createPipeline(type, factor);
     }
     TestUtils.openAllRatisPipelines(pipelineManager);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
index 7aca0f3..513b60f 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
@@ -24,6 +24,7 @@
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
 import org.apache.hadoop.hdds.scm.net.Node;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
@@ -35,6 +36,7 @@
 import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.NodeReportProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -173,14 +175,28 @@
     this.safemode = safemode;
   }
 
+
   /**
    * Gets all Live Datanodes that is currently communicating with SCM.
    *
-   * @param nodestate - State of the node
+   * @param status The status of the node
    * @return List of Datanodes that are Heartbeating SCM.
    */
   @Override
-  public List<DatanodeDetails> getNodes(HddsProtos.NodeState nodestate) {
+  public List<DatanodeDetails> getNodes(NodeStatus status) {
+    return getNodes(status.getOperationalState(), status.getHealth());
+  }
+
+  /**
+   * Gets all Live Datanodes that is currently communicating with SCM.
+   *
+   * @param opState - The operational State of the node
+   * @param nodestate - The health of the node
+   * @return List of Datanodes that are Heartbeating SCM.
+   */
+  @Override
+  public List<DatanodeDetails> getNodes(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState nodestate) {
     if (nodestate == HEALTHY) {
       return healthyNodes;
     }
@@ -199,12 +215,24 @@
   /**
    * Returns the Number of Datanodes that are communicating with SCM.
    *
+   * @param status - Status of the node
+   * @return int -- count
+   */
+  @Override
+  public int getNodeCount(NodeStatus status) {
+    return getNodeCount(status.getOperationalState(), status.getHealth());
+  }
+
+  /**
+   * Returns the Number of Datanodes that are communicating with SCM.
+   *
    * @param nodestate - State of the node
    * @return int -- count
    */
   @Override
-  public int getNodeCount(HddsProtos.NodeState nodestate) {
-    List<DatanodeDetails> nodes = getNodes(nodestate);
+  public int getNodeCount(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState nodestate) {
+    List<DatanodeDetails> nodes = getNodes(opState, nodestate);
     if (nodes != null) {
       return nodes.size();
     }
@@ -261,11 +289,31 @@
    * @return Healthy/Stale/Dead.
    */
   @Override
-  public HddsProtos.NodeState getNodeState(DatanodeDetails dd) {
+  public NodeStatus getNodeStatus(DatanodeDetails dd)
+      throws NodeNotFoundException {
     return null;
   }
 
   /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  public void setNodeOperationalState(DatanodeDetails datanodeDetails,
+      HddsProtos.NodeOperationalState newState) throws NodeNotFoundException {
+  }
+
+  /**
+   * Set the operation state of a node.
+   * @param datanodeDetails The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  public void setNodeOperationalState(DatanodeDetails datanodeDetails,
+      HddsProtos.NodeOperationalState newState, long opStateExpiryEpocSec)
+      throws NodeNotFoundException {
+  }
+
+  /**
    * Get set of pipelines a datanode is part of.
    * @param dnId - datanodeID
    * @return Set of PipelineID
@@ -498,12 +546,23 @@
   }
 
   @Override
-  public Map<String, Integer> getNodeCount() {
-    Map<String, Integer> nodeCountMap = new HashMap<String, Integer>();
-    for (HddsProtos.NodeState state : HddsProtos.NodeState.values()) {
-      nodeCountMap.put(state.toString(), getNodeCount(state));
+  public Map<String, Map<String, Integer>> getNodeCount() {
+    Map<String, Map<String, Integer>> nodes = new HashMap<>();
+    for (NodeOperationalState opState : NodeOperationalState.values()) {
+      Map<String, Integer> states = new HashMap<>();
+      for (HddsProtos.NodeState health : HddsProtos.NodeState.values()) {
+        states.put(health.name(), 0);
+      }
+      nodes.put(opState.name(), states);
     }
-    return nodeCountMap;
+    // At the moment MockNodeManager is not aware of decommission and
+    // maintenance states, therefore loop over all nodes and assume all nodes
+    // are IN_SERVICE. This will be fixed as part of HDDS-2673
+    for (HddsProtos.NodeState state : HddsProtos.NodeState.values()) {
+      nodes.get(NodeOperationalState.IN_SERVICE.name())
+          .compute(state.name(), (k, v) -> v + 1);
+    }
+    return nodes;
   }
 
   @Override
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java
new file mode 100644
index 0000000..4cabd01
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/SimpleMockNodeManager.java
@@ -0,0 +1,323 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
+import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.ozone.protocol.VersionResponse;
+import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
+import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Basic implementation of the NodeManager interface which can be used in tests.
+ *
+ * TODO - Merge the functionality with MockNodeManager, as it needs refactored
+ *        after the introduction of decommission and maintenance states.
+ */
+public class SimpleMockNodeManager implements NodeManager {
+
+  private Map<UUID, DatanodeInfo> nodeMap = new ConcurrentHashMap<>();
+  private Map<UUID, Set<PipelineID>> pipelineMap = new ConcurrentHashMap<>();
+  private Map<UUID, Set<ContainerID>> containerMap = new ConcurrentHashMap<>();
+
+  public void register(DatanodeDetails dd, NodeStatus status) {
+    dd.setPersistedOpState(status.getOperationalState());
+    dd.setPersistedOpStateExpiryEpochSec(status.getOpStateExpiryEpochSeconds());
+    nodeMap.put(dd.getUuid(), new DatanodeInfo(dd, status));
+  }
+
+  public void setNodeStatus(DatanodeDetails dd, NodeStatus status) {
+    dd.setPersistedOpState(status.getOperationalState());
+    dd.setPersistedOpStateExpiryEpochSec(status.getOpStateExpiryEpochSeconds());
+    DatanodeInfo dni = nodeMap.get(dd.getUuid());
+    dni.setNodeStatus(status);
+  }
+
+  /**
+   * Set the number of pipelines for the given node. This simply generates
+   * new PipelineID objects and places them in a set. No actual pipelines are
+   * created.
+   *
+   * Setting the count to zero effectively deletes the pipelines for the node
+   *
+   * @param dd The DatanodeDetails for which to create the pipelines
+   * @param count The number of pipelines to create or zero to delete all
+   *              pipelines
+   */
+  public void setPipelines(DatanodeDetails dd, int count) {
+    Set<PipelineID> pipelines = new HashSet<>();
+    for (int i=0; i<count; i++) {
+      pipelines.add(PipelineID.randomId());
+    }
+    pipelineMap.put(dd.getUuid(), pipelines);
+  }
+
+  /**
+   * If the given node was registed with the nodeManager, return the
+   * NodeStatus for the node. Otherwise return a NodeStatus of "In Service
+   * and Healthy".
+   * @param datanodeDetails DatanodeDetails
+   * @return The NodeStatus of the node if it is registered, otherwise an
+   *         Inservice and Healthy NodeStatus.
+   */
+  @Override
+  public NodeStatus getNodeStatus(DatanodeDetails datanodeDetails)
+      throws NodeNotFoundException {
+    DatanodeInfo dni = nodeMap.get(datanodeDetails.getUuid());
+    if (dni != null) {
+      return dni.getNodeStatus();
+    } else {
+      return NodeStatus.inServiceHealthy();
+    }
+  }
+
+  @Override
+  public void setNodeOperationalState(DatanodeDetails dn,
+      HddsProtos.NodeOperationalState newState) throws NodeNotFoundException {
+    setNodeOperationalState(dn, newState, 0);
+  }
+
+  @Override
+  public void setNodeOperationalState(DatanodeDetails dn,
+      HddsProtos.NodeOperationalState newState, long opStateExpiryEpocSec)
+      throws NodeNotFoundException {
+    DatanodeInfo dni = nodeMap.get(dn.getUuid());
+    if (dni == null) {
+      throw new NodeNotFoundException();
+    }
+    dni.setNodeStatus(
+        new NodeStatus(
+            newState, dni.getNodeStatus().getHealth(), opStateExpiryEpocSec));
+  }
+
+  /**
+   * Return the set of PipelineID associated with the given DatanodeDetails.
+   *
+   * If there are no pipelines, null is return, to mirror the behaviour of
+   * SCMNodeManager.
+   *
+   * @param datanodeDetails The datanode for which to return the pipelines
+   * @return A set of PipelineID or null if there are none
+   */
+  @Override
+  public Set<PipelineID> getPipelines(DatanodeDetails datanodeDetails) {
+    Set<PipelineID> p = pipelineMap.get(datanodeDetails.getUuid());
+    if (p == null || p.size() == 0) {
+      return null;
+    } else {
+      return p;
+    }
+  }
+
+  @Override
+  public int getPipelinesCount(DatanodeDetails datanodeDetails) {
+    return 0;
+  }
+
+  @Override
+  public void setContainers(DatanodeDetails dn,
+      Set<ContainerID> containerIds) throws NodeNotFoundException {
+    containerMap.put(dn.getUuid(), containerIds);
+  }
+
+  /**
+   * Return the set of ContainerID associated with the datanode. If there are no
+   * container present, an empty set is return to mirror the behaviour of
+   * SCMNodeManaer
+   *
+   * @param dn The datanodeDetails for which to return the containers
+   * @return A Set of ContainerID or an empty Set if none are present
+   * @throws NodeNotFoundException
+   */
+  @Override
+  public Set<ContainerID> getContainers(DatanodeDetails dn)
+      throws NodeNotFoundException {
+    // The concrete implementation of this method in SCMNodeManager will return
+    // an empty set if there are no containers, and will never return null.
+    return containerMap
+        .computeIfAbsent(dn.getUuid(), key -> new HashSet<>());
+  }
+
+  /**
+   * Below here, are all auto-generate placeholder methods to implement the
+   * interface.
+   */
+
+  @Override
+  public List<DatanodeDetails> getNodes(NodeStatus nodeStatus) {
+    return null;
+  }
+
+  @Override
+  public List<DatanodeDetails> getNodes(
+      HddsProtos.NodeOperationalState opState, HddsProtos.NodeState health) {
+    return null;
+  }
+
+  @Override
+  public int getNodeCount(NodeStatus nodeStatus) {
+    return 0;
+  }
+
+  @Override
+  public int getNodeCount(HddsProtos.NodeOperationalState opState,
+                          HddsProtos.NodeState health) {
+    return 0;
+  }
+
+  @Override
+  public List<DatanodeDetails> getAllNodes() {
+    return null;
+  }
+
+  @Override
+  public SCMNodeStat getStats() {
+    return null;
+  }
+
+  @Override
+  public Map<DatanodeDetails, SCMNodeStat> getNodeStats() {
+    return null;
+  }
+
+  @Override
+  public SCMNodeMetric getNodeStat(DatanodeDetails datanodeDetails) {
+    return null;
+  }
+
+  @Override
+  public void addPipeline(Pipeline pipeline) {
+  }
+
+  @Override
+  public void removePipeline(Pipeline pipeline) {
+  }
+
+  @Override
+  public void addContainer(DatanodeDetails datanodeDetails,
+      ContainerID containerId) throws NodeNotFoundException {
+  }
+
+
+
+  @Override
+  public void addDatanodeCommand(UUID dnId, SCMCommand command) {
+  }
+
+  @Override
+  public void processNodeReport(DatanodeDetails datanodeDetails,
+      StorageContainerDatanodeProtocolProtos.NodeReportProto nodeReport) {
+  }
+
+  @Override
+  public List<SCMCommand> getCommandQueue(UUID dnID) {
+    return null;
+  }
+
+  @Override
+  public DatanodeDetails getNodeByUuid(String uuid) {
+    return null;
+  }
+
+  @Override
+  public List<DatanodeDetails> getNodesByAddress(String address) {
+    return null;
+  }
+
+  @Override
+  public NetworkTopology getClusterNetworkTopologyMap() {
+    return null;
+  }
+
+  @Override
+  public int minHealthyVolumeNum(List<DatanodeDetails> dnList) {
+    return 0;
+  }
+
+  @Override
+  public int pipelineLimit(DatanodeDetails dn) {
+    return 1;
+  }
+
+  @Override
+  public int minPipelineLimit(List<DatanodeDetails> dn) {
+    return 0;
+  }
+
+  @Override
+  public void close() throws IOException {
+
+  }
+
+  @Override
+  public Map<String, Map<String, Integer>> getNodeCount() {
+    return null;
+  }
+
+  @Override
+  public Map<String, Long> getNodeInfo() {
+    return null;
+  }
+
+  @Override
+  public void onMessage(CommandForDatanode commandForDatanode,
+                        EventPublisher publisher) {
+  }
+
+  @Override
+  public VersionResponse getVersion(
+      StorageContainerDatanodeProtocolProtos.SCMVersionRequestProto
+          versionRequest) {
+    return null;
+  }
+
+  @Override
+  public RegisteredCommand register(DatanodeDetails datanodeDetails,
+      StorageContainerDatanodeProtocolProtos.NodeReportProto nodeReport,
+      StorageContainerDatanodeProtocolProtos.PipelineReportsProto
+      pipelineReport) {
+    return null;
+  }
+
+  @Override
+  public List<SCMCommand> processHeartbeat(DatanodeDetails datanodeDetails) {
+    return null;
+  }
+
+  @Override
+  public Boolean isNodeRegistered(DatanodeDetails datanodeDetails) {
+    return null;
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
index 205fea8..979e37f 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerReportHandler.java
@@ -20,7 +20,6 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
@@ -28,6 +27,7 @@
     .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.scm.server
     .SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
@@ -116,7 +116,7 @@
     final ContainerReportHandler reportHandler = new ContainerReportHandler(
         nodeManager, containerManager);
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
     final DatanodeDetails datanodeThree = nodeIterator.next();
@@ -185,7 +185,7 @@
         nodeManager, containerManager);
 
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
     final DatanodeDetails datanodeThree = nodeIterator.next();
@@ -264,7 +264,7 @@
         nodeManager, containerManager);
 
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
     final DatanodeDetails datanodeThree = nodeIterator.next();
@@ -343,7 +343,7 @@
         nodeManager, containerManager);
 
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
     final DatanodeDetails datanodeThree = nodeIterator.next();
@@ -420,7 +420,7 @@
     final ContainerReportHandler reportHandler = new ContainerReportHandler(
         nodeManager, containerManager);
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
 
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
@@ -491,7 +491,7 @@
     final ContainerReportHandler reportHandler = new ContainerReportHandler(
         nodeManager, containerManager);
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
 
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
@@ -562,7 +562,7 @@
     final ContainerReportHandler reportHandler = new ContainerReportHandler(
         nodeManager, containerManager);
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
 
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final DatanodeDetails datanodeTwo = nodeIterator.next();
@@ -635,7 +635,7 @@
         nodeManager, containerManager);
 
     final Iterator<DatanodeDetails> nodeIterator = nodeManager.getNodes(
-        NodeState.HEALTHY).iterator();
+        NodeStatus.inServiceHealthy()).iterator();
     final DatanodeDetails datanodeOne = nodeIterator.next();
     final ContainerInfo containerOne = getContainer(LifeCycleState.DELETED);
 
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestReplicationManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestReplicationManager.java
index b11582a..887850f 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestReplicationManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestReplicationManager.java
@@ -18,21 +18,24 @@
 
 package org.apache.hadoop.hdds.scm.container;
 
+import com.google.common.primitives.Longs;
 import org.apache.hadoop.hdds.conf.ConfigurationSource;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.SCMCommandProto;
-import org.apache.hadoop.hdds.scm.container.ReplicationManager.ReplicationManagerConfiguration;
+import org.apache.hadoop.hdds.scm.container.ReplicationManager
+    .ReplicationManagerConfiguration;
 import org.apache.hadoop.hdds.scm.PlacementPolicy;
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.server.events.EventHandler;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
 import org.apache.hadoop.hdds.server.events.EventQueue;
@@ -59,6 +62,13 @@
 import java.util.stream.IntStream;
 
 import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.createDatanodeDetails;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSED;
 import static org.apache.hadoop.hdds.scm.TestUtils.getContainer;
 import static org.apache.hadoop.hdds.scm.TestUtils.getReplicas;
 import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
@@ -73,13 +83,17 @@
   private PlacementPolicy containerPlacementPolicy;
   private EventQueue eventQueue;
   private DatanodeCommandHandler datanodeCommandHandler;
+  private SimpleMockNodeManager nodeManager;
+  private ContainerManager containerManager;
+  private ConfigurationSource conf;
   private SCMNodeManager scmNodeManager;
 
   @Before
-  public void setup() throws IOException, InterruptedException {
-    final ConfigurationSource conf = new OzoneConfiguration();
-    final ContainerManager containerManager =
-        Mockito.mock(ContainerManager.class);
+  public void setup()
+      throws IOException, InterruptedException, NodeNotFoundException {
+    conf = new OzoneConfiguration();
+    containerManager = Mockito.mock(ContainerManager.class);
+    nodeManager = new SimpleMockNodeManager();
     eventQueue = new EventQueue();
     containerStateManager = new ContainerStateManager(conf);
 
@@ -119,9 +133,9 @@
         });
 
     scmNodeManager = Mockito.mock(SCMNodeManager.class);
-    Mockito.when(scmNodeManager.getNodeState(
+    Mockito.when(scmNodeManager.getNodeStatus(
         Mockito.any(DatanodeDetails.class)))
-        .thenReturn(NodeState.HEALTHY);
+        .thenReturn(NodeStatus.inServiceHealthy());
 
     replicationManager = new ReplicationManager(
         new ReplicationManagerConfiguration(),
@@ -129,7 +143,21 @@
         containerPlacementPolicy,
         eventQueue,
         new LockManager<>(conf),
-        scmNodeManager);
+        nodeManager);
+    replicationManager.start();
+    Thread.sleep(100L);
+  }
+
+  private void createReplicationManager(ReplicationManagerConfiguration rmConf)
+      throws InterruptedException {
+    replicationManager = new ReplicationManager(
+        rmConf,
+        containerManager,
+        containerPlacementPolicy,
+        eventQueue,
+        new LockManager<ContainerID>(conf),
+        nodeManager);
+
     replicationManager.start();
     Thread.sleep(100L);
   }
@@ -594,7 +622,7 @@
       throws SCMException, ContainerNotFoundException, InterruptedException {
     final ContainerInfo container = getContainer(LifeCycleState.CLOSED);
     final ContainerID id = container.containerID();
-    final Set<ContainerReplica> replicas = getReplicas(id, State.CLOSED,
+    final Set<ContainerReplica> replicas = getReplicas(id, CLOSED,
         randomDatanodeDetails(),
         randomDatanodeDetails(),
         randomDatanodeDetails());
@@ -843,6 +871,243 @@
         .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand));
   }
 
+  /**
+   * ReplicationManager should replicate an additional replica if there are
+   * decommissioned replicas.
+   */
+  @Test
+  public void testUnderReplicatedDueToDecommission() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    assertReplicaScheduled(2);
+  }
+
+  /**
+   * ReplicationManager should replicate an additional replica when all copies
+   * are decommissioning.
+   */
+  @Test
+  public void testUnderReplicatedDueToAllDecommission() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    assertReplicaScheduled(3);
+  }
+
+  /**
+   * ReplicationManager should not take any action when the container is
+   * correctly replicated with decommissioned replicas still present.
+   */
+  @Test
+  public void testCorrectlyReplicatedWithDecommission() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONING, HEALTHY), CLOSED);
+    assertReplicaScheduled(0);
+  }
+
+  /**
+   * ReplicationManager should replicate an additional replica when min rep
+   * is not met for maintenance.
+   */
+  @Test
+  public void testUnderReplicatedDueToMaintenance() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(1);
+  }
+
+  /**
+   * ReplicationManager should not replicate an additional replica when if
+   * min replica for maintenance is 1 and another replica is available.
+   */
+  @Test
+  public void testNotUnderReplicatedDueToMaintenanceMinRepOne() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    replicationManager.stop();
+    ReplicationManagerConfiguration newConf =
+        new ReplicationManagerConfiguration();
+    newConf.setMaintenanceReplicaMinimum(1);
+    createReplicationManager(newConf);
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(0);
+  }
+
+  /**
+   * ReplicationManager should replicate an additional replica when all copies
+   * are going off line and min rep is 1.
+   */
+  @Test
+  public void testUnderReplicatedDueToMaintenanceMinRepOne() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    replicationManager.stop();
+    ReplicationManagerConfiguration newConf =
+        new ReplicationManagerConfiguration();
+    newConf.setMaintenanceReplicaMinimum(1);
+    createReplicationManager(newConf);
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(1);
+  }
+
+  /**
+   * ReplicationManager should replicate additional replica when all copies
+   * are going into maintenance.
+   */
+  @Test
+  public void testUnderReplicatedDueToAllMaintenance() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(2);
+  }
+
+  /**
+   * ReplicationManager should not replicate additional replica sufficient
+   * replica are available.
+   */
+  @Test
+  public void testCorrectlyReplicatedWithMaintenance() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_SERVICE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(0);
+  }
+
+  /**
+   * ReplicationManager should replicate additional replica when all copies
+   * are decommissioning or maintenance.
+   */
+  @Test
+  public void testUnderReplicatedWithDecommissionAndMaintenance() throws
+      SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    assertReplicaScheduled(2);
+  }
+
+  /**
+   * When a CLOSED container is over replicated, ReplicationManager
+   * deletes the excess replicas. While choosing the replica for deletion
+   * ReplicationManager should not attempt to remove a DECOMMISSION or
+   * MAINTENANCE replica.
+   */
+  @Test
+  public void testOverReplicatedClosedContainerWithDecomAndMaint()
+      throws SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONED, HEALTHY), CLOSED);
+    addReplica(container, new NodeStatus(IN_MAINTENANCE, HEALTHY), CLOSED);
+    addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
+    addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
+    addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
+    addReplica(container, NodeStatus.inServiceHealthy(), CLOSED);
+
+    final int currentDeleteCommandCount = datanodeCommandHandler
+        .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand);
+
+    replicationManager.processContainersNow();
+    // Wait for EventQueue to call the event handler
+    Thread.sleep(100L);
+    Assert.assertEquals(currentDeleteCommandCount + 2, datanodeCommandHandler
+        .getInvocationCount(SCMCommandProto.Type.deleteContainerCommand));
+    // Get the DECOM and Maint replica and ensure none of them are scheduled
+    // for removal
+    Set<ContainerReplica> decom =
+        containerStateManager.getContainerReplicas(container.containerID())
+        .stream()
+        .filter(r -> r.getDatanodeDetails().getPersistedOpState() != IN_SERVICE)
+        .collect(Collectors.toSet());
+    for (ContainerReplica r : decom) {
+      Assert.assertFalse(datanodeCommandHandler.received(
+          SCMCommandProto.Type.deleteContainerCommand,
+          r.getDatanodeDetails()));
+    }
+  }
+
+  /**
+   * Replication Manager should not attempt to replicate from an unhealthy
+   * (stale or dead) node. To test this, setup a scenario where a replia needs
+   * to be created, but mark all nodes stale. That way, no new replica will be
+   * scheduled.
+   */
+  @Test
+  public void testUnderReplicatedNotHealthySource()
+      throws SCMException, ContainerNotFoundException, InterruptedException {
+    final ContainerInfo container = createContainer(LifeCycleState.CLOSED);
+    addReplica(container, NodeStatus.inServiceStale(), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONED, STALE), CLOSED);
+    addReplica(container, new NodeStatus(DECOMMISSIONED, STALE), CLOSED);
+    // There should be replica scheduled, but as all nodes are stale, nothing
+    // gets scheduled.
+    assertReplicaScheduled(0);
+  }
+
+  private ContainerInfo createContainer(LifeCycleState containerState)
+      throws SCMException {
+    final ContainerInfo container = getContainer(containerState);
+    final ContainerID id = container.containerID();
+    containerStateManager.loadContainer(container);
+    return container;
+  }
+
+  private ContainerReplica addReplica(ContainerInfo container,
+      NodeStatus nodeStatus, State replicaState)
+      throws ContainerNotFoundException {
+    DatanodeDetails dn = randomDatanodeDetails();
+    dn.setPersistedOpState(nodeStatus.getOperationalState());
+    dn.setPersistedOpStateExpiryEpochSec(
+        nodeStatus.getOpStateExpiryEpochSeconds());
+    nodeManager.register(dn, nodeStatus);
+    // Using the same originID for all replica in the container set. If each
+    // replica has a unique originID, it causes problems in ReplicationManager
+    // when processing over-replicated containers.
+    final UUID originNodeId =
+        UUID.nameUUIDFromBytes(Longs.toByteArray(container.getContainerID()));
+    final ContainerReplica replica = getReplicas(
+        container.containerID(), CLOSED, 1000L, originNodeId, dn);
+    containerStateManager
+        .updateContainerReplica(container.containerID(), replica);
+    return replica;
+  }
+
+  private void assertReplicaScheduled(int delta) throws InterruptedException {
+    final int currentReplicateCommandCount = datanodeCommandHandler
+        .getInvocationCount(SCMCommandProto.Type.replicateContainerCommand);
+
+    replicationManager.processContainersNow();
+    // Wait for EventQueue to call the event handler
+    Thread.sleep(100L);
+    Assert.assertEquals(currentReplicateCommandCount + delta,
+        datanodeCommandHandler.getInvocationCount(
+            SCMCommandProto.Type.replicateContainerCommand));
+  }
+
   @After
   public void teardown() throws IOException {
     containerStateManager.close();
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestUnknownContainerReport.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestUnknownContainerReport.java
index 1c2cdd0..f2e4968 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestUnknownContainerReport.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestUnknownContainerReport.java
@@ -27,7 +27,6 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -35,6 +34,7 @@
 import org.apache.hadoop.hdds.scm.ScmConfig;
 import org.apache.hadoop.hdds.scm.events.SCMEvents;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.server
     .SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -107,7 +107,7 @@
 
     ContainerInfo container = getContainer(LifeCycleState.CLOSED);
     Iterator<DatanodeDetails> nodeIterator = nodeManager
-        .getNodes(NodeState.HEALTHY).iterator();
+        .getNodes(NodeStatus.inServiceHealthy()).iterator();
     DatanodeDetails datanode = nodeIterator.next();
 
     ContainerReportsProto containerReport = getContainerReportsProto(
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
index 842c494..aa506cb 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestContainerPlacementFactory.java
@@ -23,7 +23,6 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
 import org.apache.hadoop.hdds.scm.PlacementPolicy;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
@@ -34,15 +33,17 @@
 import org.apache.hadoop.hdds.scm.net.NodeSchema;
 import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
 
 import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
 import static org.apache.hadoop.hdds.scm.net.NetConstants.RACK_SCHEMA;
 import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+
 import static org.mockito.Matchers.anyObject;
-import org.mockito.Mockito;
 import static org.mockito.Mockito.when;
 
 /**
@@ -89,7 +90,7 @@
 
     // create mock node manager
     nodeManager = Mockito.mock(NodeManager.class);
-    when(nodeManager.getNodes(NodeState.HEALTHY))
+    when(nodeManager.getNodes(NodeStatus.inServiceHealthy()))
         .thenReturn(new ArrayList<>(datanodes));
     when(nodeManager.getNodeStat(anyObject()))
         .thenReturn(new SCMNodeMetric(storageCapacity, 0L, 100L));
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
index afefc9a..ee9c029 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementCapacity.java
@@ -25,11 +25,11 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.junit.Assert;
 import org.junit.Test;
 import static org.mockito.Matchers.anyObject;
@@ -51,7 +51,7 @@
     }
 
     NodeManager mockNodeManager = Mockito.mock(NodeManager.class);
-    when(mockNodeManager.getNodes(NodeState.HEALTHY))
+    when(mockNodeManager.getNodes(NodeStatus.inServiceHealthy()))
         .thenReturn(new ArrayList<>(datanodes));
 
     when(mockNodeManager.getNodeStat(anyObject()))
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
index 5019ed4..1c332b7 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRackAware.java
@@ -25,7 +25,6 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -35,6 +34,13 @@
 import org.apache.hadoop.hdds.scm.net.NodeSchema;
 import org.apache.hadoop.hdds.scm.net.NodeSchemaManager;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
 
 import org.apache.commons.lang3.StringUtils;
 import static org.apache.hadoop.hdds.scm.net.NetConstants.LEAF_SCHEMA;
@@ -42,18 +48,12 @@
 import static org.apache.hadoop.hdds.scm.net.NetConstants.ROOT_SCHEMA;
 import org.hamcrest.MatcherAssert;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
-import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 import static org.mockito.Matchers.anyObject;
-import org.mockito.Mockito;
 import static org.mockito.Mockito.when;
 
 /**
@@ -107,10 +107,10 @@
 
     // create mock node manager
     nodeManager = Mockito.mock(NodeManager.class);
+    when(nodeManager.getNodes(NodeStatus.inServiceHealthy()))
+        .thenReturn(new ArrayList<>(datanodes));
     when(nodeManager.getClusterNetworkTopologyMap())
         .thenReturn(cluster);
-    when(nodeManager.getNodes(NodeState.HEALTHY))
-        .thenReturn(new ArrayList<>(datanodes));
     when(nodeManager.getNodeStat(anyObject()))
         .thenReturn(new SCMNodeMetric(STORAGE_CAPACITY, 0L, 100L));
     if (datanodeCount > 4) {
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
index fb8d2e0..416c3f2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/placement/algorithms/TestSCMContainerPlacementRandom.java
@@ -23,12 +23,12 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.ContainerPlacementStatus;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.junit.Assert;
 import org.junit.Test;
 import static junit.framework.TestCase.assertEquals;
@@ -54,7 +54,7 @@
     }
 
     NodeManager mockNodeManager = Mockito.mock(NodeManager.class);
-    when(mockNodeManager.getNodes(NodeState.HEALTHY))
+    when(mockNodeManager.getNodes(NodeStatus.inServiceHealthy()))
         .thenReturn(new ArrayList<>(datanodes));
 
     when(mockNodeManager.getNodeStat(anyObject()))
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/TestContainerReplicaCount.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/TestContainerReplicaCount.java
new file mode 100644
index 0000000..3c7c952
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/states/TestContainerReplicaCount.java
@@ -0,0 +1,465 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.container.states;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ContainerReplicaCount;
+import org.junit.Before;
+import org.junit.Test;
+import java.util.*;
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .NodeOperationalState.ENTERING_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos
+    .NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSED;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.OPEN;
+import static org.junit.Assert.assertFalse;
+
+/**
+ * Class used to test the ContainerReplicaCount class.
+ */
+public class TestContainerReplicaCount {
+
+  @Before
+  public void setup() {
+  }
+
+  @Test
+  public void testThreeHealthyReplica() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testTwoHealthyReplica() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testOneHealthyReplica() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 2, false);
+  }
+
+  @Test
+  public void testTwoHealthyAndInflightAdd() {
+
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 3, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  /**
+   * This does not schedule a container to be removed, as the inFlight add may
+   * fail and then the delete would make things under-replicated. Once the add
+   * completes there will be 4 healthy and it will get taken care of then.
+   */
+  public void testThreeHealthyAndInflightAdd() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 3, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  /**
+   * As the inflight delete may fail, but as it will make the the container
+   * under replicated, we go ahead and schedule another replica to be added.
+   */
+  public void testThreeHealthyAndInflightDelete() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 1, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  /**
+   * This is NOT sufficiently replicated as the inflight add may fail and the
+   * inflight del could succeed, leaving only 2 healthy replicas.
+   */
+  public void testThreeHealthyAndInflightAddAndInFlightDelete() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 1, 3, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  public void testFourHealthyReplicas() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, true, -1, true);
+  }
+
+  @Test
+  public void testFourHealthyReplicasAndInFlightDelete() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 1, 3, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testFourHealthyReplicasAndTwoInFlightDelete() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 2, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testOneHealthyReplicaRepFactorOne() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 1, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testOneHealthyReplicaRepFactorOneInFlightDelete() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 1, 1, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testTwoHealthyReplicaTwoInflightAdd() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 2, 0, 3, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  /**
+   * From here consider decommission replicas.
+   */
+
+  @Test
+  public void testThreeHealthyAndTwoDecommission() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE, IN_SERVICE,
+        IN_SERVICE, DECOMMISSIONING, DECOMMISSIONING);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testOneDecommissionedReplica() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, DECOMMISSIONING);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testTwoHealthyOneDecommissionedneInFlightAdd() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, DECOMMISSIONED);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 3, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  public void testAllDecommissioned() {
+    Set<ContainerReplica> replica =
+        registerNodes(DECOMMISSIONED, DECOMMISSIONED, DECOMMISSIONED);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 3, false);
+  }
+
+  @Test
+  public void testAllDecommissionedRepFactorOne() {
+    Set<ContainerReplica> replica = registerNodes(DECOMMISSIONED);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 1, 2);
+    validate(rcnt, false, 1, false);
+
+  }
+
+  @Test
+  public void testAllDecommissionedRepFactorOneInFlightAdd() {
+    Set<ContainerReplica> replica = registerNodes(DECOMMISSIONED);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 1, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  public void testOneHealthyOneDecommissioningRepFactorOne() {
+    Set<ContainerReplica> replica = registerNodes(DECOMMISSIONED, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 1, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  /**
+   * Maintenance tests from here.
+   */
+
+  @Test
+  public void testOneHealthyTwoMaintenanceMinRepOfTwo() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_MAINTENANCE, IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testOneHealthyThreeMaintenanceMinRepOfTwo() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE,
+        IN_MAINTENANCE, IN_MAINTENANCE, ENTERING_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testOneHealthyTwoMaintenanceMinRepOfOne() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_MAINTENANCE, ENTERING_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 1);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testOneHealthyThreeMaintenanceMinRepOfTwoInFlightAdd() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE,
+        IN_MAINTENANCE, ENTERING_MAINTENANCE, IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 3, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  public void testAllMaintenance() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_MAINTENANCE, ENTERING_MAINTENANCE, IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, false, 2, false);
+  }
+
+  @Test
+  /**
+   * As we have exactly 3 healthy, but then an excess of maintenance copies
+   * we ignore the over-replication caused by the maintenance copies until they
+   * come back online, and then deal with them.
+   */
+  public void testThreeHealthyTwoInMaintenance() {
+    Set<ContainerReplica> replica = registerNodes(IN_SERVICE, IN_SERVICE,
+        IN_SERVICE, IN_MAINTENANCE, ENTERING_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  /**
+   * This is somewhat similar to testThreeHealthyTwoInMaintenance() except now
+   * one of the maintenance copies has become healthy and we will need to remove
+   * the over-replicated healthy container.
+   */
+  public void testFourHealthyOneInMaintenance() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE, IN_SERVICE,
+            IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    validate(rcnt, true, -1, true);
+  }
+
+  @Test
+  public void testOneMaintenanceMinRepOfTwoRepFactorOne() {
+    Set<ContainerReplica> replica = registerNodes(IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 1, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testOneMaintenanceMinRepOfTwoRepFactorOneInFlightAdd() {
+    Set<ContainerReplica> replica = registerNodes(IN_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 1, 2);
+    validate(rcnt, false, 0, false);
+  }
+
+  @Test
+  public void testOneHealthyOneMaintenanceRepFactorOne() {
+    Set<ContainerReplica> replica = registerNodes(IN_MAINTENANCE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 1, 2);
+    validate(rcnt, true, 0, false);
+  }
+
+  @Test
+  public void testTwoDecomTwoMaintenanceOneInflightAdd() {
+    Set<ContainerReplica> replica =
+        registerNodes(DECOMMISSIONED, DECOMMISSIONING,
+            IN_MAINTENANCE, ENTERING_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 1, 0, 3, 2);
+    validate(rcnt, false, 1, false);
+  }
+
+  @Test
+  public void testHealthyContainerIsHealthy() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    assertTrue(rcnt.isHealthy());
+  }
+
+  @Test
+  public void testIsHealthyWithDifferentReplicaStateNotHealthy() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_SERVICE);
+    for (ContainerReplica r : replica) {
+      DatanodeDetails dn = r.getDatanodeDetails();
+
+      ContainerReplica replace = new ContainerReplica.ContainerReplicaBuilder()
+          .setContainerID(new ContainerID(1))
+          .setContainerState(OPEN)
+          .setDatanodeDetails(dn)
+          .setOriginNodeId(dn.getUuid())
+          .setSequenceId(1)
+          .build();
+      replica.remove(r);
+      replica.add(replace);
+      break;
+    }
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    assertFalse(rcnt.isHealthy());
+  }
+
+  @Test
+  public void testIsHealthyWithMaintReplicaIsHealthy() {
+    Set<ContainerReplica> replica =
+        registerNodes(IN_SERVICE, IN_SERVICE, IN_MAINTENANCE,
+            ENTERING_MAINTENANCE);
+    ContainerInfo container = createContainer(HddsProtos.LifeCycleState.CLOSED);
+    ContainerReplicaCount rcnt =
+        new ContainerReplicaCount(container, replica, 0, 0, 3, 2);
+    assertTrue(rcnt.isHealthy());
+  }
+
+  private void validate(ContainerReplicaCount rcnt,
+      boolean sufficientlyReplicated, int replicaDelta,
+      boolean overReplicated) {
+    assertEquals(sufficientlyReplicated, rcnt.isSufficientlyReplicated());
+    assertEquals(overReplicated, rcnt.isOverReplicated());
+    assertEquals(replicaDelta, rcnt.additionalReplicaNeeded());
+  }
+
+  private Set<ContainerReplica> registerNodes(
+      HddsProtos.NodeOperationalState... states) {
+    Set<ContainerReplica> replica = new HashSet<>();
+    for (HddsProtos.NodeOperationalState s : states) {
+      DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
+      dn.setPersistedOpState(s);
+      replica.add(new ContainerReplica.ContainerReplicaBuilder()
+          .setContainerID(new ContainerID(1))
+          .setContainerState(CLOSED)
+          .setDatanodeDetails(dn)
+          .setOriginNodeId(dn.getUuid())
+          .setSequenceId(1)
+          .build());
+    }
+    return replica;
+  }
+
+  private ContainerInfo createContainer(HddsProtos.LifeCycleState state) {
+    return new ContainerInfo.Builder()
+        .setContainerID(new ContainerID(1).getId())
+        .setState(state)
+        .build();
+  }
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
index 797709e..37a18b6 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestContainerPlacement.java
@@ -160,7 +160,7 @@
 
       //TODO: wait for heartbeat to be processed
       Thread.sleep(4 * 1000);
-      assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(nodeCount, nodeManager.getNodeCount(null, HEALTHY));
       assertEquals(capacity * nodeCount,
           (long) nodeManager.getStats().getCapacity().get());
       assertEquals(used * nodeCount,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java
new file mode 100644
index 0000000..33fe35f
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDatanodeAdminMonitor.java
@@ -0,0 +1,530 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto
+    .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
+import org.apache.hadoop.hdds.scm.container.*;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.server.events.EventHandler;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.hdds.server.events.EventQueue;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto.State.CLOSED;
+import static org.mockito.Mockito.reset;
+
+/**
+ * Tests to ensure the DatanodeAdminMonitor is working correctly. This class
+ * uses mocks or basic implementations of the key classes outside of the
+ * datanodeAdminMonitor to allow it to be tested in isolation.
+ */
+public class TestDatanodeAdminMonitor {
+
+  private SimpleMockNodeManager nodeManager;
+  private OzoneConfiguration conf;
+  private DatanodeAdminMonitorImpl monitor;
+  private DatanodeAdminHandler startAdminHandler;
+  private ReplicationManager repManager;
+  private EventQueue eventQueue;
+
+  @Before
+  public void setup() throws IOException, AuthenticationException {
+    conf = new OzoneConfiguration();
+
+    eventQueue = new EventQueue();
+    startAdminHandler = new DatanodeAdminHandler();
+    eventQueue.addHandler(SCMEvents.START_ADMIN_ON_NODE, startAdminHandler);
+
+    nodeManager = new SimpleMockNodeManager();
+
+    repManager = Mockito.mock(ReplicationManager.class);
+
+    monitor =
+        new DatanodeAdminMonitorImpl(conf, eventQueue, nodeManager, repManager);
+  }
+
+  @After
+  public void teardown() {
+  }
+
+  @Test
+  public void testNodeCanBeQueuedAndCancelled() {
+    DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
+    monitor.startMonitoring(dn);
+    assertEquals(1, monitor.getPendingCount());
+
+    monitor.stopMonitoring(dn);
+    assertEquals(0, monitor.getPendingCount());
+    assertEquals(1, monitor.getCancelledCount());
+
+    monitor.startMonitoring(dn);
+    assertEquals(1, monitor.getPendingCount());
+    assertEquals(0, monitor.getCancelledCount());
+  }
+
+  /**
+   * In this test we ensure there are some pipelines for the node being
+   * decommissioned, but there are no containers. Therefore the workflow
+   * must wait until the pipelines have closed before completing the flow.
+   */
+  @Test
+  public void testClosePipelinesEventFiredWhenAdminStarted()
+      throws NodeNotFoundException{
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.HEALTHY));
+    // Ensure the node has some pipelines
+    nodeManager.setPipelines(dn1, 2);
+    // Add the node to the monitor
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    // Ensure a StartAdmin event was fired
+    eventQueue.processAll(20000);
+    assertEquals(1, startAdminHandler.getInvocation());
+    // Ensure a node is now tracked for decommission
+    assertEquals(1, monitor.getTrackedNodeCount());
+    // Ensure the node remains decommissioning
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+    // Run the monitor again, and it should remain decommissioning
+    monitor.run();
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+
+    // Clear the pipelines and the node should transition to DECOMMISSIONED
+    nodeManager.setPipelines(dn1, 0);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(DECOMMISSIONED,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  /**
+   * In this test, there are no open pipelines and no containers on the node.
+   * Therefore, we expect the decommission flow to finish on the first run
+   * on the monitor, leaving zero nodes tracked and the node in DECOMMISSIONED
+   * state.
+   */
+  @Test
+  public void testDecommissionNodeTransitionsToCompleteWhenNoContainers()
+      throws NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.HEALTHY));
+
+    // Add the node to the monitor. By default we have zero pipelines and
+    // zero containers in the test setup, so the node should immediately
+    // transition to COMPLETED state
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    NodeStatus newStatus = nodeManager.getNodeStatus(dn1);
+    assertEquals(DECOMMISSIONED,
+        newStatus.getOperationalState());
+  }
+
+  @Test
+  public void testDecommissionNodeWaitsForContainersToReplicate()
+      throws NodeNotFoundException, ContainerNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.HEALTHY));
+
+    nodeManager.setContainers(dn1, generateContainers(3));
+    // Mock Replication Manager to return ContainerReplicaCount's which
+    // always have a DECOMMISSIONED replica.
+    mockGetContainerReplicaCount(
+        HddsProtos.LifeCycleState.CLOSED,
+        DECOMMISSIONED,
+        IN_SERVICE,
+        IN_SERVICE);
+
+    // Run the monitor for the first time and the node will transition to
+    // REPLICATE_CONTAINERS as there are no pipelines to close.
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    DatanodeDetails node = getFirstTrackedNode();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+
+    // Running the monitor again causes it to remain DECOMMISSIONING
+    // as nothing has changed.
+    monitor.run();
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+
+    // Now change the replicationManager mock to return 3 CLOSED replicas
+    // and the node should complete the REPLICATE_CONTAINERS step, moving to
+    // complete which will end the decommission workflow
+    mockGetContainerReplicaCount(
+        HddsProtos.LifeCycleState.CLOSED,
+        IN_SERVICE,
+        IN_SERVICE,
+        IN_SERVICE);
+
+    monitor.run();
+
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONED,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testDecommissionAbortedWhenNodeInUnexpectedState()
+      throws NodeNotFoundException, ContainerNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.HEALTHY));
+
+    nodeManager.setContainers(dn1, generateContainers(3));
+    mockGetContainerReplicaCount(
+        HddsProtos.LifeCycleState.CLOSED,
+        DECOMMISSIONED,
+        IN_SERVICE,
+        IN_SERVICE);
+
+    // Add the node to the monitor, it should have 3 under-replicated containers
+    // after the first run
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+
+    // Set the node to dead, and then the workflow should get aborted, setting
+    // the node state back to IN_SERVICE on the next run.
+    nodeManager.setNodeStatus(dn1,
+        new NodeStatus(IN_SERVICE,
+            HddsProtos.NodeState.HEALTHY));
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testDecommissionAbortedWhenNodeGoesDead()
+      throws NodeNotFoundException, ContainerNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.HEALTHY));
+
+    nodeManager.setContainers(dn1, generateContainers(3));
+    mockGetContainerReplicaCount(
+        HddsProtos.LifeCycleState.CLOSED,
+        DECOMMISSIONED, IN_SERVICE, IN_SERVICE);
+
+    // Add the node to the monitor, it should have 3 under-replicated containers
+    // after the first run
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+
+    // Set the node to dead, and then the workflow should get aborted, setting
+    // the node state back to IN_SERVICE.
+    nodeManager.setNodeStatus(dn1,
+        new NodeStatus(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+            HddsProtos.NodeState.DEAD));
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testMaintenanceWaitsForMaintenanceToComplete()
+      throws NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(ENTERING_MAINTENANCE,
+            HddsProtos.NodeState.HEALTHY));
+
+    // Add the node to the monitor, it should transiting to
+    // IN_MAINTENANCE as there are no containers to replicate.
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertTrue(nodeManager.getNodeStatus(dn1).isInMaintenance());
+
+    // Running the monitor again causes the node to remain in maintenance
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    assertTrue(nodeManager.getNodeStatus(dn1).isInMaintenance());
+
+    // Set the maintenance end time to a time in the past and then the node
+    // should complete the workflow and transition to IN_SERVICE
+    nodeManager.setNodeOperationalState(node,
+        HddsProtos.NodeOperationalState.IN_MAINTENANCE, -1);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testMaintenanceEndsClosingPipelines()
+      throws NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(ENTERING_MAINTENANCE,
+            HddsProtos.NodeState.HEALTHY));
+    // Ensure the node has some pipelines
+    nodeManager.setPipelines(dn1, 2);
+    // Add the node to the monitor
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    DatanodeDetails node = getFirstTrackedNode();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    assertTrue(nodeManager.getNodeStatus(dn1).isEnteringMaintenance());
+
+    // Set the maintenance end time to the past and the node should complete
+    // the workflow and return to IN_SERVICE
+    nodeManager.setNodeOperationalState(node,
+        HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE, -1);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testMaintenanceEndsWhileReplicatingContainers()
+      throws ContainerNotFoundException, NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(ENTERING_MAINTENANCE,
+            HddsProtos.NodeState.HEALTHY));
+
+    nodeManager.setContainers(dn1, generateContainers(3));
+    mockGetContainerReplicaCount(
+        HddsProtos.LifeCycleState.CLOSED,
+        IN_MAINTENANCE,
+        ENTERING_MAINTENANCE,
+        IN_MAINTENANCE);
+
+    // Add the node to the monitor, it should transiting to
+    // REPLICATE_CONTAINERS as the containers are under-replicated for
+    // maintenance.
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertTrue(nodeManager.getNodeStatus(dn1).isEnteringMaintenance());
+
+    nodeManager.setNodeOperationalState(node,
+        HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE, -1);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  @Test
+  public void testDeadMaintenanceNodeDoesNotAbortWorkflow()
+      throws NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(ENTERING_MAINTENANCE,
+            HddsProtos.NodeState.HEALTHY));
+
+    // Add the node to the monitor, it should transiting to
+    // AWAIT_MAINTENANCE_END as there are no under-replicated containers.
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertTrue(nodeManager.getNodeStatus(dn1).isInMaintenance());
+
+    // Set the node dead and ensure the workflow does not end
+    NodeStatus status = nodeManager.getNodeStatus(dn1);
+    nodeManager.setNodeStatus(dn1, new NodeStatus(
+        status.getOperationalState(), HddsProtos.NodeState.DEAD));
+
+    // Running the monitor again causes the node to remain in maintenance
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    assertTrue(nodeManager.getNodeStatus(dn1).isInMaintenance());
+  }
+
+  @Test
+  public void testCancelledNodesMovedToInService()
+      throws NodeNotFoundException {
+    DatanodeDetails dn1 = MockDatanodeDetails.randomDatanodeDetails();
+    nodeManager.register(dn1,
+        new NodeStatus(ENTERING_MAINTENANCE,
+            HddsProtos.NodeState.HEALTHY));
+
+    // Add the node to the monitor, it should transiting to
+    // AWAIT_MAINTENANCE_END as there are no under-replicated containers.
+    monitor.startMonitoring(dn1);
+    monitor.run();
+    assertEquals(1, monitor.getTrackedNodeCount());
+    DatanodeDetails node = getFirstTrackedNode();
+    assertTrue(nodeManager.getNodeStatus(dn1).isInMaintenance());
+
+    // Now cancel the node and run the monitor, the node should be IN_SERVICE
+    monitor.stopMonitoring(dn1);
+    monitor.run();
+    assertEquals(0, monitor.getTrackedNodeCount());
+    assertEquals(IN_SERVICE,
+        nodeManager.getNodeStatus(dn1).getOperationalState());
+  }
+
+  /**
+   * Generate a set of ContainerID, starting from an ID of zero up to the given
+   * count minus 1.
+   * @param count The number of ContainerID objects to generate.
+   * @return A Set of ContainerID objects.
+   */
+  private Set<ContainerID> generateContainers(int count) {
+    Set<ContainerID> containers = new HashSet<>();
+    for (int i=0; i<count; i++) {
+      containers.add(new ContainerID(i));
+    }
+    return containers;
+  }
+
+  /**
+   * Create a ContainerReplicaCount object, including a container with the
+   * requested ContainerID and state, along with a set of replicas of the given
+   * states.
+   * @param containerID The ID of the container to create an included
+   * @param containerState The state of the container
+   * @param states Create a replica for each of the given states.
+   * @return A ContainerReplicaCount containing the generated container and
+   *         replica set
+   */
+  private ContainerReplicaCount generateReplicaCount(ContainerID containerID,
+      HddsProtos.LifeCycleState containerState,
+      HddsProtos.NodeOperationalState...states) {
+    Set<ContainerReplica> replicas = new HashSet<>();
+    for (HddsProtos.NodeOperationalState s : states) {
+      replicas.add(generateReplica(containerID, s, CLOSED));
+    }
+    ContainerInfo container = new ContainerInfo.Builder()
+        .setContainerID(containerID.getId())
+        .setState(containerState)
+        .build();
+
+    return new ContainerReplicaCount(container, replicas, 0, 0, 3, 2);
+  }
+
+  /**
+   * Generate a new ContainerReplica with the given containerID and State.
+   * @param containerID The ID the replica is associated with
+   * @param nodeState The persistedOpState stored in datanodeDetails.
+   * @param replicaState The state of the generated replica.
+   * @return A containerReplica with the given ID and state
+   */
+  private ContainerReplica generateReplica(ContainerID containerID,
+      HddsProtos.NodeOperationalState nodeState,
+      ContainerReplicaProto.State replicaState) {
+    DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
+    dn.setPersistedOpState(nodeState);
+    return ContainerReplica.newBuilder()
+        .setContainerState(replicaState)
+        .setContainerID(containerID)
+        .setSequenceId(1)
+        .setDatanodeDetails(dn)
+        .build();
+  }
+
+  /**
+   * Helper method to get the first node from the set of trackedNodes within
+   * the monitor.
+   * @return DatanodeAdminNodeDetails for the first tracked node found.
+   */
+  private DatanodeDetails getFirstTrackedNode() {
+    return
+        monitor.getTrackedNodes().toArray(new DatanodeDetails[0])[0];
+  }
+
+  /**
+   * The only interaction the DatanodeAdminMonitor has with the
+   * ReplicationManager, is to request a ContainerReplicaCount object for each
+   * container on nodes being deocmmissioned or moved to maintenance. This
+   * method mocks that interface to return a ContainerReplicaCount with a
+   * container in the given containerState and a set of replias in the given
+   * replicaStates.
+   * @param containerState
+   * @param replicaStates
+   * @throws ContainerNotFoundException
+   */
+  private void mockGetContainerReplicaCount(
+      HddsProtos.LifeCycleState containerState,
+      HddsProtos.NodeOperationalState...replicaStates)
+      throws ContainerNotFoundException {
+    reset(repManager);
+    Mockito.when(repManager.getContainerReplicaCount(
+        Mockito.any(ContainerID.class)))
+        .thenAnswer(invocation ->
+            generateReplicaCount((ContainerID)invocation.getArguments()[0],
+                containerState, replicaStates));
+  }
+
+  /**
+   * This simple internal class is used to track and handle any DatanodeAdmin
+   * events fired by the DatanodeAdminMonitor during tests.
+   */
+  private class DatanodeAdminHandler implements
+      EventHandler<DatanodeDetails> {
+
+    private AtomicInteger invocation = new AtomicInteger(0);
+
+    @Override
+    public void onMessage(final DatanodeDetails dn,
+                          final EventPublisher publisher) {
+      invocation.incrementAndGet();
+    }
+
+    public int getInvocation() {
+      return invocation.get();
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java
index f05be76..2611d10 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestDeadNodeHandler.java
@@ -35,6 +35,7 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
     .StorageContainerDatanodeProtocolProtos.ContainerReplicaProto;
 import org.apache.hadoop.hdds.protocol.proto
@@ -191,22 +192,44 @@
     TestUtils.closeContainer(containerManager, container2.containerID());
     TestUtils.quasiCloseContainer(containerManager, container3.containerID());
 
+    // First set the node to IN_MAINTENANCE and ensure the container replicas
+    // are not removed on the dead event
+    nodeManager.setNodeOperationalState(datanode1,
+        HddsProtos.NodeOperationalState.IN_MAINTENANCE);
     deadNodeHandler.onMessage(datanode1, publisher);
 
     Set<ContainerReplica> container1Replicas = containerManager
         .getContainerReplicas(new ContainerID(container1.getContainerID()));
+    Assert.assertEquals(2, container1Replicas.size());
+
+    Set<ContainerReplica> container2Replicas = containerManager
+        .getContainerReplicas(new ContainerID(container2.getContainerID()));
+    Assert.assertEquals(2, container2Replicas.size());
+
+    Set<ContainerReplica> container3Replicas = containerManager
+            .getContainerReplicas(new ContainerID(container3.getContainerID()));
+    Assert.assertEquals(1, container3Replicas.size());
+
+    // Now set the node to anything other than IN_MAINTENANCE and the relevant
+    // replicas should be removed
+    nodeManager.setNodeOperationalState(datanode1,
+        HddsProtos.NodeOperationalState.IN_SERVICE);
+    deadNodeHandler.onMessage(datanode1, publisher);
+
+    container1Replicas = containerManager
+        .getContainerReplicas(new ContainerID(container1.getContainerID()));
     Assert.assertEquals(1, container1Replicas.size());
     Assert.assertEquals(datanode2,
         container1Replicas.iterator().next().getDatanodeDetails());
 
-    Set<ContainerReplica> container2Replicas = containerManager
+    container2Replicas = containerManager
         .getContainerReplicas(new ContainerID(container2.getContainerID()));
     Assert.assertEquals(1, container2Replicas.size());
     Assert.assertEquals(datanode2,
         container2Replicas.iterator().next().getDatanodeDetails());
 
-    Set<ContainerReplica> container3Replicas = containerManager
-            .getContainerReplicas(new ContainerID(container3.getContainerID()));
+    container3Replicas = containerManager
+        .getContainerReplicas(new ContainerID(container3.getContainerID()));
     Assert.assertEquals(1, container3Replicas.size());
     Assert.assertEquals(datanode3,
         container3Replicas.iterator().next().getDatanodeDetails());
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeDecommissionManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeDecommissionManager.java
new file mode 100644
index 0000000..25d8744
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeDecommissionManager.java
@@ -0,0 +1,297 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.node;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.HddsTestUtils;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Before;
+import org.junit.Test;
+import java.io.IOException;
+import java.util.List;
+import java.util.UUID;
+import java.util.Arrays;
+import java.util.ArrayList;
+import static junit.framework.TestCase.assertEquals;
+import static org.assertj.core.api.Fail.fail;
+import static org.junit.Assert.assertNotEquals;
+
+/**
+ * Unit tests for the decommision manager.
+ */
+
+public class TestNodeDecommissionManager {
+
+  private NodeDecommissionManager decom;
+  private StorageContainerManager scm;
+  private NodeManager nodeManager;
+  private OzoneConfiguration conf;
+  private String storageDir;
+
+  @Before
+  public void setup() throws Exception {
+    conf = new OzoneConfiguration();
+    storageDir = GenericTestUtils.getTempPath(
+        TestDeadNodeHandler.class.getSimpleName() + UUID.randomUUID());
+    conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
+    nodeManager = createNodeManager(conf);
+    decom = new NodeDecommissionManager(
+        conf, nodeManager, null, null, null);
+  }
+
+  @Test
+  public void testHostStringsParseCorrectly()
+      throws InvalidHostStringException {
+    NodeDecommissionManager.HostDefinition def =
+        new NodeDecommissionManager.HostDefinition("foobar");
+    assertEquals("foobar", def.getHostname());
+    assertEquals(-1, def.getPort());
+
+    def = new NodeDecommissionManager.HostDefinition(" foobar ");
+    assertEquals("foobar", def.getHostname());
+    assertEquals(-1, def.getPort());
+
+    def = new NodeDecommissionManager.HostDefinition("foobar:1234");
+    assertEquals("foobar", def.getHostname());
+    assertEquals(1234, def.getPort());
+
+    def = new NodeDecommissionManager.HostDefinition(
+        "foobar.mycompany.com:1234");
+    assertEquals("foobar.mycompany.com", def.getHostname());
+    assertEquals(1234, def.getPort());
+
+    try {
+      def = new NodeDecommissionManager.HostDefinition("foobar:abcd");
+      fail("InvalidHostStringException should have been thrown");
+    } catch (InvalidHostStringException e) {
+    }
+  }
+
+  @Test
+  public void testAnyInvalidHostThrowsException()
+      throws InvalidHostStringException{
+    List<DatanodeDetails> dns = generateDatanodes();
+
+    // Try to decommission a host that does exist, but give incorrect port
+    try {
+      decom.decommissionNodes(Arrays.asList(dns.get(1).getIpAddress()+":10"));
+      fail("InvalidHostStringException expected");
+    } catch (InvalidHostStringException e) {
+    }
+
+    // Try to decommission a host that does not exist
+    try {
+      decom.decommissionNodes(Arrays.asList("123.123.123.123"));
+      fail("InvalidHostStringException expected");
+    } catch (InvalidHostStringException e) {
+    }
+
+    // Try to decommission a host that does exist and a host that does not
+    try {
+      decom.decommissionNodes(Arrays.asList(
+          dns.get(1).getIpAddress(), "123,123,123,123"));
+      fail("InvalidHostStringException expected");
+    } catch (InvalidHostStringException e) {
+    }
+
+    // Try to decommission a host with many DNs on the address with no port
+    try {
+      decom.decommissionNodes(Arrays.asList(
+          dns.get(0).getIpAddress()));
+      fail("InvalidHostStringException expected");
+    } catch (InvalidHostStringException e) {
+    }
+
+    // Try to decommission a host with many DNs on the address with a port
+    // that does not exist
+    try {
+      decom.decommissionNodes(Arrays.asList(
+          dns.get(0).getIpAddress()+":10"));
+      fail("InvalidHostStringException expected");
+    } catch (InvalidHostStringException e) {
+    }
+  }
+
+  @Test
+  public void testNodesCanBeDecommissionedAndRecommissioned()
+      throws InvalidHostStringException, NodeNotFoundException {
+    List<DatanodeDetails> dns = generateDatanodes();
+
+    // Decommission 2 valid nodes
+    decom.decommissionNodes(Arrays.asList(dns.get(1).getIpAddress(),
+        dns.get(2).getIpAddress()));
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+
+    // Running the command again gives no error - nodes already decommissioning
+    // are silently ignored.
+    decom.decommissionNodes(Arrays.asList(dns.get(1).getIpAddress(),
+        dns.get(2).getIpAddress()));
+
+    // Attempt to decommission dn(10) which has multiple hosts on the same IP
+    // and we hardcoded ports to 3456, 4567, 5678
+    DatanodeDetails multiDn = dns.get(10);
+    String multiAddr =
+        multiDn.getIpAddress()+":"+multiDn.getPorts().get(0).getValue();
+    decom.decommissionNodes(Arrays.asList(multiAddr));
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(multiDn).getOperationalState());
+
+    // Recommission all 3 hosts
+    decom.recommissionNodes(Arrays.asList(
+        multiAddr, dns.get(1).getIpAddress(), dns.get(2).getIpAddress()));
+    decom.getMonitor().run();
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(10)).getOperationalState());
+  }
+
+  @Test
+  public void testNodesCanBePutIntoMaintenanceAndRecommissioned()
+      throws InvalidHostStringException, NodeNotFoundException {
+    List<DatanodeDetails> dns = generateDatanodes();
+
+    // Put 2 valid nodes into maintenance
+    decom.startMaintenanceNodes(Arrays.asList(dns.get(1).getIpAddress(),
+        dns.get(2).getIpAddress()), 100);
+    assertEquals(HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertNotEquals(0, nodeManager.getNodeStatus(
+        dns.get(1)).getOpStateExpiryEpochSeconds());
+    assertEquals(HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+    assertNotEquals(0, nodeManager.getNodeStatus(
+        dns.get(2)).getOpStateExpiryEpochSeconds());
+
+    // Running the command again gives no error - nodes already decommissioning
+    // are silently ignored.
+    decom.startMaintenanceNodes(Arrays.asList(dns.get(1).getIpAddress(),
+        dns.get(2).getIpAddress()), 100);
+
+    // Attempt to decommission dn(10) which has multiple hosts on the same IP
+    // and we hardcoded ports to 3456, 4567, 5678
+    DatanodeDetails multiDn = dns.get(10);
+    String multiAddr =
+        multiDn.getIpAddress()+":"+multiDn.getPorts().get(0).getValue();
+    decom.startMaintenanceNodes(Arrays.asList(multiAddr), 100);
+    assertEquals(HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE,
+        nodeManager.getNodeStatus(multiDn).getOperationalState());
+
+    // Recommission all 3 hosts
+    decom.recommissionNodes(Arrays.asList(
+        multiAddr, dns.get(1).getIpAddress(), dns.get(2).getIpAddress()));
+    decom.getMonitor().run();
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.IN_SERVICE,
+        nodeManager.getNodeStatus(dns.get(10)).getOperationalState());
+  }
+
+  @Test
+  public void testNodesCannotTransitionFromDecomToMaint() throws Exception {
+    List<DatanodeDetails> dns = generateDatanodes();
+
+    // Put 1 node into maintenance and another into decom
+    decom.startMaintenance(dns.get(1), 100);
+    decom.startDecommission(dns.get(2));
+    assertEquals(HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+
+    // Try to go from maint to decom:
+    try {
+      decom.startDecommission(dns.get(1));
+      fail("Expected InvalidNodeStateException");
+    } catch (InvalidNodeStateException e) {
+    }
+
+    // Try to go from decom to maint:
+    try {
+      decom.startMaintenance(dns.get(2), 100);
+      fail("Expected InvalidNodeStateException");
+    } catch (InvalidNodeStateException e) {
+    }
+
+    // Ensure the states are still as before
+    assertEquals(HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE,
+        nodeManager.getNodeStatus(dns.get(1)).getOperationalState());
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONING,
+        nodeManager.getNodeStatus(dns.get(2)).getOperationalState());
+  }
+
+
+
+  private SCMNodeManager createNodeManager(OzoneConfiguration config)
+      throws IOException, AuthenticationException {
+    scm = HddsTestUtils.getScm(config);
+    return (SCMNodeManager) scm.getScmNodeManager();
+  }
+
+  /**
+   * Generate a list of random DNs and return the list. A total of 11 DNs will
+   * be generated and registered with the node manager. Index 0 and 10 will
+   * have the same IP and host and the rest will have unique IPs and Hosts.
+   * The DN at index 10, has 3 hard coded ports of 3456, 4567, 5678. All other
+   * DNs will have ports set to 0.
+   * @return The list of DatanodeDetails Generated
+   */
+  private List<DatanodeDetails> generateDatanodes() {
+    List<DatanodeDetails> dns = new ArrayList<>();
+    for (int i=0; i<10; i++) {
+      DatanodeDetails dn = MockDatanodeDetails.randomDatanodeDetails();
+      dns.add(dn);
+      nodeManager.register(dn, null, null);
+    }
+    // We have 10 random DNs, we want to create another one that is on the same
+    // host as some of the others.
+    DatanodeDetails multiDn = dns.get(0);
+
+    DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
+    builder.setUuid(UUID.randomUUID())
+        .setHostName(multiDn.getHostName())
+        .setIpAddress(multiDn.getIpAddress())
+        .addPort(DatanodeDetails.newPort(
+            DatanodeDetails.Port.Name.STANDALONE, 3456))
+        .addPort(DatanodeDetails.newPort(
+            DatanodeDetails.Port.Name.RATIS, 4567))
+        .addPort(DatanodeDetails.newPort(
+            DatanodeDetails.Port.Name.REST, 5678))
+        .setNetworkLocation(multiDn.getNetworkLocation());
+
+    DatanodeDetails dn = builder.build();
+    nodeManager.register(dn, null, null);
+    dns.add(dn);
+    return dns;
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java
new file mode 100644
index 0000000..e19f626
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestNodeStateManager.java
@@ -0,0 +1,308 @@
+/**
+ * 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.node;
+
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.hdds.scm.events.SCMEvents;
+import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.server.events.Event;
+import org.apache.hadoop.hdds.server.events.EventPublisher;
+import org.apache.hadoop.util.Time;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertNull;
+
+/**
+ * Class to test the NodeStateManager, which is an internal class used by
+ * the SCMNodeManager.
+ */
+
+public class TestNodeStateManager {
+
+  private NodeStateManager nsm;
+  private ConfigurationSource conf;
+  private MockEventPublisher eventPublisher;
+
+  @Before
+  public void setUp() {
+    conf = new ConfigurationSource() {
+      @Override
+      public String get(String key) {
+        return null;
+      }
+
+      @Override
+      public Collection<String> getConfigKeys() {
+        return null;
+      }
+
+      @Override
+      public char[] getPassword(String key) throws IOException {
+        return new char[0];
+      }
+    };
+    eventPublisher = new MockEventPublisher();
+    nsm = new NodeStateManager(conf, eventPublisher);
+  }
+
+  @After
+  public void tearDown() {
+  }
+
+  @Test
+  public void testNodeCanBeAddedAndRetrieved()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    // Create a datanode, then add and retrieve it
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+    assertEquals(dn.getUuid(), nsm.getNode(dn).getUuid());
+    // Now get the status of the newly added node and it should be
+    // IN_SERVICE and HEALTHY
+    NodeStatus expectedState = NodeStatus.inServiceHealthy();
+    assertEquals(expectedState, nsm.getNodeStatus(dn));
+  }
+
+  @Test
+  public void testGetAllNodesReturnsCorrectly()
+      throws NodeAlreadyExistsException {
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+    dn = generateDatanode();
+    nsm.addNode(dn);
+    assertEquals(2, nsm.getAllNodes().size());
+    assertEquals(2, nsm.getTotalNodeCount());
+  }
+
+  @Test
+  public void testGetNodeCountReturnsCorrectly()
+      throws NodeAlreadyExistsException {
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+    assertEquals(1, nsm.getNodes(NodeStatus.inServiceHealthy()).size());
+    assertEquals(0, nsm.getNodes(NodeStatus.inServiceStale()).size());
+  }
+
+  @Test
+  public void testGetNodeCount() throws NodeAlreadyExistsException {
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+    assertEquals(1, nsm.getNodeCount(NodeStatus.inServiceHealthy()));
+    assertEquals(0, nsm.getNodeCount(NodeStatus.inServiceStale()));
+  }
+
+  @Test
+  public void testNodesMarkedDeadAndStale()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    long now = Time.monotonicNow();
+
+    // Set the dead and stale limits to be 1 second larger than configured
+    long staleLimit = HddsServerUtil.getStaleNodeInterval(conf) + 1000;
+    long deadLimit = HddsServerUtil.getDeadNodeInterval(conf) + 1000;
+
+    DatanodeDetails staleDn = generateDatanode();
+    nsm.addNode(staleDn);
+    nsm.getNode(staleDn).updateLastHeartbeatTime(now - staleLimit);
+
+    DatanodeDetails deadDn = generateDatanode();
+    nsm.addNode(deadDn);
+    nsm.getNode(deadDn).updateLastHeartbeatTime(now - deadLimit);
+
+    DatanodeDetails healthyDn = generateDatanode();
+    nsm.addNode(healthyDn);
+    nsm.getNode(healthyDn).updateLastHeartbeatTime();
+
+    nsm.checkNodesHealth();
+    assertEquals(healthyDn, nsm.getHealthyNodes().get(0));
+    // A node cannot go directly to dead. It must be marked stale first
+    // due to the allowed state transitions. Therefore we will initially have 2
+    // stale nodesCheck it is in stale nodes
+    assertEquals(2, nsm.getStaleNodes().size());
+    // Now check health again and it should be in deadNodes()
+    nsm.checkNodesHealth();
+    assertEquals(staleDn, nsm.getStaleNodes().get(0));
+    assertEquals(deadDn, nsm.getDeadNodes().get(0));
+  }
+
+  @Test
+  public void testNodeCanTransitionThroughHealthStatesAndFiresEvents()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    long now = Time.monotonicNow();
+
+    // Set the dead and stale limits to be 1 second larger than configured
+    long staleLimit = HddsServerUtil.getStaleNodeInterval(conf) + 1000;
+    long deadLimit = HddsServerUtil.getDeadNodeInterval(conf) + 1000;
+
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+    assertEquals(SCMEvents.NEW_NODE, eventPublisher.getLastEvent());
+    DatanodeInfo dni = nsm.getNode(dn);
+    dni.updateLastHeartbeatTime();
+
+    // Ensure node is initially healthy
+    eventPublisher.clearEvents();
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.HEALTHY, nsm.getNodeStatus(dn).getHealth());
+    assertNull(eventPublisher.getLastEvent());
+
+    // Set the heartbeat old enough to make it stale
+    dni.updateLastHeartbeatTime(now - staleLimit);
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.STALE, nsm.getNodeStatus(dn).getHealth());
+    assertEquals(SCMEvents.STALE_NODE, eventPublisher.getLastEvent());
+
+    // Now make it dead
+    dni.updateLastHeartbeatTime(now - deadLimit);
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.DEAD, nsm.getNodeStatus(dn).getHealth());
+    assertEquals(SCMEvents.DEAD_NODE, eventPublisher.getLastEvent());
+
+    // Transition back to healthy from dead
+    dni.updateLastHeartbeatTime();
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.HEALTHY, nsm.getNodeStatus(dn).getHealth());
+    assertEquals(SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE,
+        eventPublisher.getLastEvent());
+
+    // Make the node stale again, and transition to healthy.
+    dni.updateLastHeartbeatTime(now - staleLimit);
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.STALE, nsm.getNodeStatus(dn).getHealth());
+    assertEquals(SCMEvents.STALE_NODE, eventPublisher.getLastEvent());
+    dni.updateLastHeartbeatTime();
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.HEALTHY, nsm.getNodeStatus(dn).getHealth());
+    assertEquals(SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE,
+        eventPublisher.getLastEvent());
+  }
+
+  @Test
+  public void testNodeOpStateCanBeSet()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+
+    nsm.setNodeOperationalState(dn,
+        HddsProtos.NodeOperationalState.DECOMMISSIONED);
+
+    NodeStatus newStatus = nsm.getNodeStatus(dn);
+    assertEquals(HddsProtos.NodeOperationalState.DECOMMISSIONED,
+        newStatus.getOperationalState());
+    assertEquals(NodeState.HEALTHY,
+        newStatus.getHealth());
+  }
+
+  @Test
+  public void testHealthEventsFiredWhenOpStateChanged()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    DatanodeDetails dn = generateDatanode();
+    nsm.addNode(dn);
+
+    // First set the node to decommissioned, then run through all op states in
+    // order and ensure the non_healthy_to_healthy event gets fired
+    nsm.setNodeOperationalState(dn,
+        HddsProtos.NodeOperationalState.DECOMMISSIONED);
+    for (HddsProtos.NodeOperationalState s :
+        HddsProtos.NodeOperationalState.values()) {
+      eventPublisher.clearEvents();
+      nsm.setNodeOperationalState(dn, s);
+      assertEquals(SCMEvents.NON_HEALTHY_TO_HEALTHY_NODE,
+          eventPublisher.getLastEvent());
+    }
+
+    // Now make the node stale and run through all states again ensuring the
+    // stale event gets fired
+    long now = Time.monotonicNow();
+    long staleLimit = HddsServerUtil.getStaleNodeInterval(conf) + 1000;
+    long deadLimit = HddsServerUtil.getDeadNodeInterval(conf) + 1000;
+    DatanodeInfo dni = nsm.getNode(dn);
+    dni.updateLastHeartbeatTime(now - staleLimit);
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.STALE, nsm.getNodeStatus(dn).getHealth());
+    nsm.setNodeOperationalState(dn,
+        HddsProtos.NodeOperationalState.DECOMMISSIONED);
+    for (HddsProtos.NodeOperationalState s :
+        HddsProtos.NodeOperationalState.values()) {
+      eventPublisher.clearEvents();
+      nsm.setNodeOperationalState(dn, s);
+      assertEquals(SCMEvents.STALE_NODE, eventPublisher.getLastEvent());
+    }
+
+    // Finally make the node dead and run through all the op states again
+    dni.updateLastHeartbeatTime(now - deadLimit);
+    nsm.checkNodesHealth();
+    assertEquals(NodeState.DEAD, nsm.getNodeStatus(dn).getHealth());
+    nsm.setNodeOperationalState(dn,
+        HddsProtos.NodeOperationalState.DECOMMISSIONED);
+    for (HddsProtos.NodeOperationalState s :
+        HddsProtos.NodeOperationalState.values()) {
+      eventPublisher.clearEvents();
+      nsm.setNodeOperationalState(dn, s);
+      assertEquals(SCMEvents.DEAD_NODE, eventPublisher.getLastEvent());
+    }
+  }
+
+  private DatanodeDetails generateDatanode() {
+    return DatanodeDetails.newBuilder().setUuid(UUID.randomUUID()).build();
+  }
+
+  static class  MockEventPublisher implements EventPublisher {
+
+    private List<Event> events = new ArrayList<>();
+    private List<Object> payloads = new ArrayList<>();
+
+    public void clearEvents() {
+      events.clear();
+      payloads.clear();
+    }
+
+    public List<Event> getEvents() {
+      return events;
+    }
+
+    public Event getLastEvent() {
+      if (events.size() == 0) {
+        return null;
+      } else {
+        return events.get(events.size()-1);
+      }
+    }
+
+    @Override
+    public <PAYLOAD, EVENT_TYPE extends Event<PAYLOAD>> void
+        fireEvent(EVENT_TYPE event, PAYLOAD payload) {
+      events.add(event);
+      payloads.add(payload);
+    }
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
index 3f3c4ae..5eaeedb 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
@@ -31,9 +31,9 @@
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdds.DFSConfigKeysLegacy;
 import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
@@ -43,6 +43,7 @@
 import org.apache.hadoop.hdds.scm.TestUtils;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -50,9 +51,20 @@
 import org.apache.hadoop.ozone.protocol.commands.CloseContainerCommand;
 import org.apache.hadoop.ozone.protocol.commands.CommandForDatanode;
 import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.ozone.protocol.commands.SetNodeOperationalStateCommand;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.Map;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static java.util.concurrent.TimeUnit.SECONDS;
@@ -61,24 +73,17 @@
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
 import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.createDatanodeDetails;
 import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_DEADNODE_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys
+    .OZONE_SCM_STALENODE_INTERVAL;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_RATIS_PIPELINE_LIMIT;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
 import static org.apache.hadoop.hdds.scm.events.SCMEvents.DATANODE_COMMAND;
-import org.junit.After;
-import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
 import org.mockito.Mockito;
 
 /**
@@ -241,7 +246,13 @@
       }
       //TODO: wait for heartbeat to be processed
       Thread.sleep(4 * 1000);
-      assertEquals(count, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(count, nodeManager.getNodeCount(
+          NodeStatus.inServiceHealthy()));
+
+      Map<String, Map<String, Integer>> nodeCounts = nodeManager.getNodeCount();
+      assertEquals(count,
+          nodeCounts.get(HddsProtos.NodeOperationalState.IN_SERVICE.name())
+              .get(HddsProtos.NodeState.HEALTHY.name()).intValue());
     }
   }
 
@@ -269,6 +280,35 @@
   }
 
   /**
+   * Ensure that a change to the operationalState of a node fires a datanode
+   * event of type SetNodeOperationalStateCommand.
+   */
+  @Test
+  @Ignore // TODO - this test is no longer valid as the heartbeat processing
+          //        now generates the command message.
+  public void testSetNodeOpStateAndCommandFired()
+      throws IOException, NodeNotFoundException, AuthenticationException {
+    final int interval = 100;
+
+    OzoneConfiguration conf = getConf();
+    conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL, interval,
+        MILLISECONDS);
+
+    try (SCMNodeManager nodeManager = createNodeManager(conf)) {
+      DatanodeDetails dn = TestUtils.createRandomDatanodeAndRegister(
+          nodeManager);
+      long expiry = System.currentTimeMillis() / 1000 + 1000;
+      nodeManager.setNodeOperationalState(dn,
+          HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE, expiry);
+      List<SCMCommand> commands = nodeManager.getCommandQueue(dn.getUuid());
+
+      Assert.assertTrue(commands.get(0).getClass().equals(
+          SetNodeOperationalStateCommand.class));
+      assertEquals(1, commands.size());
+    }
+  }
+
+  /**
    * Asserts that a single node moves from Healthy to stale node, then from
    * stale node to dead node if it misses enough heartbeats.
    *
@@ -315,15 +355,21 @@
       // Wait for 2 seconds, wait a total of 4 seconds to make sure that the
       // node moves into stale state.
       Thread.sleep(2 * 1000);
-      List<DatanodeDetails> staleNodeList = nodeManager.getNodes(STALE);
+      List<DatanodeDetails> staleNodeList =
+          nodeManager.getNodes(NodeStatus.inServiceStale());
       assertEquals("Expected to find 1 stale node",
-          1, nodeManager.getNodeCount(STALE));
+          1, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
       assertEquals("Expected to find 1 stale node",
           1, staleNodeList.size());
       assertEquals("Stale node is not the expected ID", staleNode
           .getUuid(), staleNodeList.get(0).getUuid());
       Thread.sleep(1000);
 
+      Map<String, Map<String, Integer>> nodeCounts = nodeManager.getNodeCount();
+      assertEquals(1,
+          nodeCounts.get(HddsProtos.NodeOperationalState.IN_SERVICE.name())
+              .get(HddsProtos.NodeState.STALE.name()).intValue());
+
       // heartbeat good nodes again.
       for (DatanodeDetails dn : nodeList) {
         nodeManager.processHeartbeat(dn);
@@ -334,18 +380,26 @@
       Thread.sleep(2 * 1000);
 
       // the stale node has been removed
-      staleNodeList = nodeManager.getNodes(STALE);
+      staleNodeList = nodeManager.getNodes(NodeStatus.inServiceStale());
+      nodeCounts = nodeManager.getNodeCount();
       assertEquals("Expected to find 1 stale node",
-          0, nodeManager.getNodeCount(STALE));
+          0, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
       assertEquals("Expected to find 1 stale node",
           0, staleNodeList.size());
+      assertEquals(0,
+          nodeCounts.get(HddsProtos.NodeOperationalState.IN_SERVICE.name())
+              .get(HddsProtos.NodeState.STALE.name()).intValue());
 
       // Check for the dead node now.
-      List<DatanodeDetails> deadNodeList = nodeManager.getNodes(DEAD);
+      List<DatanodeDetails> deadNodeList =
+          nodeManager.getNodes(NodeStatus.inServiceDead());
       assertEquals("Expected to find 1 dead node", 1,
-          nodeManager.getNodeCount(DEAD));
+          nodeManager.getNodeCount(NodeStatus.inServiceDead()));
       assertEquals("Expected to find 1 dead node",
           1, deadNodeList.size());
+      assertEquals(1,
+          nodeCounts.get(HddsProtos.NodeOperationalState.IN_SERVICE.name())
+              .get(HddsProtos.NodeState.DEAD.name()).intValue());
       assertEquals("Dead node is not the expected ID", staleNode
           .getUuid(), deadNodeList.get(0).getUuid());
     }
@@ -391,8 +445,8 @@
 
       //Assert all nodes are healthy.
       assertEquals(2, nodeManager.getAllNodes().size());
-      assertEquals(2, nodeManager.getNodeCount(HEALTHY));
-
+      assertEquals(2,
+          nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
       /**
        * Simulate a JVM Pause and subsequent handling in following steps:
        * Step 1 : stop heartbeat check process for stale node interval
@@ -427,7 +481,7 @@
 
       // Step 4 : all nodes should still be HEALTHY
       assertEquals(2, nodeManager.getAllNodes().size());
-      assertEquals(2, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(2, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
 
       // Step 5 : heartbeat for node1
       nodeManager.processHeartbeat(node1);
@@ -436,8 +490,8 @@
       Thread.sleep(1000);
 
       // Step 7 : node2 should transition to STALE
-      assertEquals(1, nodeManager.getNodeCount(HEALTHY));
-      assertEquals(1, nodeManager.getNodeCount(STALE));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
     }
   }
 
@@ -536,7 +590,7 @@
 
       //Assert all nodes are healthy.
       assertEquals(3, nodeManager.getAllNodes().size());
-      assertEquals(3, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(3, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
 
       /**
        * Cluster state: Quiesced: We are going to sleep for 3 seconds. Which
@@ -544,7 +598,7 @@
        */
       Thread.sleep(3 * 1000);
       assertEquals(3, nodeManager.getAllNodes().size());
-      assertEquals(3, nodeManager.getNodeCount(STALE));
+      assertEquals(3, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
 
 
       /**
@@ -562,18 +616,19 @@
       Thread.sleep(1500);
       nodeManager.processHeartbeat(healthyNode);
       Thread.sleep(2 * 1000);
-      assertEquals(1, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
 
 
       // 3.5 seconds from last heartbeat for the stale and deadNode. So those
       //  2 nodes must move to Stale state and the healthy node must
       // remain in the healthy State.
-      List<DatanodeDetails> healthyList = nodeManager.getNodes(HEALTHY);
+      List<DatanodeDetails> healthyList = nodeManager.getNodes(
+          NodeStatus.inServiceHealthy());
       assertEquals("Expected one healthy node", 1, healthyList.size());
       assertEquals("Healthy node is not the expected ID", healthyNode
           .getUuid(), healthyList.get(0).getUuid());
 
-      assertEquals(2, nodeManager.getNodeCount(STALE));
+      assertEquals(2, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
 
       /**
        * Cluster State: Allow healthyNode to remain in healthy state and
@@ -589,14 +644,16 @@
       // 3.5 seconds have elapsed for stale node, so it moves into Stale.
       // 7 seconds have elapsed for dead node, so it moves into dead.
       // 2 Seconds have elapsed for healthy node, so it stays in healthy state.
-      healthyList = nodeManager.getNodes(HEALTHY);
-      List<DatanodeDetails> staleList = nodeManager.getNodes(STALE);
-      List<DatanodeDetails> deadList = nodeManager.getNodes(DEAD);
+      healthyList = nodeManager.getNodes((NodeStatus.inServiceHealthy()));
+      List<DatanodeDetails> staleList =
+          nodeManager.getNodes(NodeStatus.inServiceStale());
+      List<DatanodeDetails> deadList =
+          nodeManager.getNodes(NodeStatus.inServiceDead());
 
       assertEquals(3, nodeManager.getAllNodes().size());
-      assertEquals(1, nodeManager.getNodeCount(HEALTHY));
-      assertEquals(1, nodeManager.getNodeCount(STALE));
-      assertEquals(1, nodeManager.getNodeCount(DEAD));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceStale()));
+      assertEquals(1, nodeManager.getNodeCount(NodeStatus.inServiceDead()));
 
       assertEquals("Expected one healthy node",
           1, healthyList.size());
@@ -622,7 +679,7 @@
       Thread.sleep(500);
       //Assert all nodes are healthy.
       assertEquals(3, nodeManager.getAllNodes().size());
-      assertEquals(3, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(3, nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
     }
   }
 
@@ -671,7 +728,7 @@
    */
   private boolean findNodes(NodeManager nodeManager, int count,
       HddsProtos.NodeState state) {
-    return count == nodeManager.getNodeCount(state);
+    return count == nodeManager.getNodeCount(NodeStatus.inServiceStale());
   }
 
   /**
@@ -744,11 +801,14 @@
       // Assert all healthy nodes are healthy now, this has to be a greater
       // than check since Stale nodes can be healthy when we check the state.
 
-      assertTrue(nodeManager.getNodeCount(HEALTHY) >= healthyCount);
+      assertTrue(nodeManager.getNodeCount(NodeStatus.inServiceHealthy())
+          >= healthyCount);
 
-      assertEquals(deadCount, nodeManager.getNodeCount(DEAD));
+      assertEquals(deadCount,
+          nodeManager.getNodeCount(NodeStatus.inServiceDead()));
 
-      List<DatanodeDetails> deadList = nodeManager.getNodes(DEAD);
+      List<DatanodeDetails> deadList =
+          nodeManager.getNodes(NodeStatus.inServiceDead());
 
       for (DatanodeDetails node : deadList) {
         assertTrue(deadNodeList.contains(node));
@@ -865,7 +925,8 @@
       //TODO: wait for EventQueue to be processed
       eventQueue.processAll(8000L);
 
-      assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(nodeCount, nodeManager.getNodeCount(
+          NodeStatus.inServiceHealthy()));
       assertEquals(capacity * nodeCount, (long) nodeManager.getStats()
           .getCapacity().get());
       assertEquals(used * nodeCount, (long) nodeManager.getStats()
@@ -915,7 +976,8 @@
       //TODO: wait for EventQueue to be processed
       eventQueue.processAll(8000L);
 
-      assertEquals(1, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(1, nodeManager
+          .getNodeCount(NodeStatus.inServiceHealthy()));
       assertEquals(volumeCount / 2,
               nodeManager.minHealthyVolumeNum(dnList));
       dnList.clear();
@@ -1008,7 +1070,7 @@
       // Wait up to 4s so that the node becomes stale
       // Verify the usage info should be unchanged.
       GenericTestUtils.waitFor(
-          () -> nodeManager.getNodeCount(STALE) == 1, 100,
+          () -> nodeManager.getNodeCount(NodeStatus.inServiceStale()) == 1, 100,
           4 * 1000);
       assertEquals(nodeCount, nodeManager.getNodeStats().size());
 
@@ -1026,7 +1088,7 @@
       // Wait up to 4 more seconds so the node becomes dead
       // Verify usage info should be updated.
       GenericTestUtils.waitFor(
-          () -> nodeManager.getNodeCount(DEAD) == 1, 100,
+          () -> nodeManager.getNodeCount(NodeStatus.inServiceDead()) == 1, 100,
           4 * 1000);
 
       assertEquals(0, nodeManager.getNodeStats().size());
@@ -1044,7 +1106,7 @@
       // Wait up to 5 seconds so that the dead node becomes healthy
       // Verify usage info should be updated.
       GenericTestUtils.waitFor(
-          () -> nodeManager.getNodeCount(HEALTHY) == 1,
+          () -> nodeManager.getNodeCount(NodeStatus.inServiceHealthy()) == 1,
           100, 5 * 1000);
       GenericTestUtils.waitFor(
           () -> nodeManager.getStats().getScmUsed().get() == expectedScmUsed,
@@ -1176,7 +1238,8 @@
       // verify network topology cluster has all the registered nodes
       Thread.sleep(4 * 1000);
       NetworkTopology clusterMap = scm.getClusterMap();
-      assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(nodeCount,
+          nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
       assertEquals(nodeCount, clusterMap.getNumOfLeafNode(""));
       assertEquals(4, clusterMap.getMaxLevel());
       List<DatanodeDetails> nodeList = nodeManager.getAllNodes();
@@ -1220,7 +1283,8 @@
       // verify network topology cluster has all the registered nodes
       Thread.sleep(4 * 1000);
       NetworkTopology clusterMap = scm.getClusterMap();
-      assertEquals(nodeCount, nodeManager.getNodeCount(HEALTHY));
+      assertEquals(nodeCount,
+          nodeManager.getNodeCount(NodeStatus.inServiceHealthy()));
       assertEquals(nodeCount, clusterMap.getNumOfLeafNode(""));
       assertEquals(3, clusterMap.getMaxLevel());
       List<DatanodeDetails> nodeList = nodeManager.getAllNodes();
@@ -1240,6 +1304,58 @@
     }
   }
 
+  @Test
+  public void testGetNodeInfo()
+      throws IOException, InterruptedException, NodeNotFoundException,
+        AuthenticationException {
+    OzoneConfiguration conf = getConf();
+    final int nodeCount = 6;
+    SCMNodeManager nodeManager = createNodeManager(conf);
+
+    for (int i=0; i<nodeCount; i++) {
+      DatanodeDetails datanodeDetails =
+          MockDatanodeDetails.randomDatanodeDetails();
+      final long capacity = 2000;
+      final long used = 100;
+      final long remaining = 1900;
+      UUID dnId = datanodeDetails.getUuid();
+      String storagePath = testDir.getAbsolutePath() + "/" + dnId;
+      StorageReportProto report = TestUtils
+          .createStorageReport(dnId, storagePath, capacity, used,
+              remaining, null);
+
+      nodeManager.register(datanodeDetails, TestUtils.createNodeReport(report),
+          TestUtils.getRandomPipelineReports());
+
+      nodeManager.processHeartbeat(datanodeDetails);
+      if (i == 5) {
+        nodeManager.setNodeOperationalState(datanodeDetails,
+            HddsProtos.NodeOperationalState.ENTERING_MAINTENANCE);
+      }
+      if (i == 3 || i == 4) {
+        nodeManager.setNodeOperationalState(datanodeDetails,
+            HddsProtos.NodeOperationalState.DECOMMISSIONED);
+      }
+    }
+    Thread.sleep(100);
+
+    Map<String, Long> stats = nodeManager.getNodeInfo();
+    // 3 IN_SERVICE nodes:
+    assertEquals(6000, stats.get("DiskCapacity").longValue());
+    assertEquals(300, stats.get("DiskUsed").longValue());
+    assertEquals(5700, stats.get("DiskRemaining").longValue());
+
+    // 2 Decommissioned nodes
+    assertEquals(4000, stats.get("DecommissionedDiskCapacity").longValue());
+    assertEquals(200, stats.get("DecommissionedDiskUsed").longValue());
+    assertEquals(3800, stats.get("DecommissionedDiskRemaining").longValue());
+
+    // 1 Maintenance node
+    assertEquals(2000, stats.get("MaintenanceDiskCapacity").longValue());
+    assertEquals(100, stats.get("MaintenanceDiskUsed").longValue());
+    assertEquals(1900, stats.get("MaintenanceDiskRemaining").longValue());
+  }
+
   /**
    * Test add node into a 4-layer network topology during node register.
    */
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNodeStateMap.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNodeStateMap.java
new file mode 100644
index 0000000..25601f7
--- /dev/null
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/states/TestNodeStateMap.java
@@ -0,0 +1,140 @@
+/**
+ * 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.node.states;
+
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.UUID;
+
+import static junit.framework.TestCase.assertEquals;
+
+/**
+ * Class to test the NodeStateMap class, which is an internal class used by
+ * NodeStateManager.
+ */
+
+public class TestNodeStateMap {
+
+  private NodeStateMap map;
+
+  @Before
+  public void setUp() {
+    map = new NodeStateMap();
+  }
+
+  @After
+  public void tearDown() {
+  }
+
+  @Test
+  public void testNodeCanBeAddedAndRetrieved()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    DatanodeDetails dn = generateDatanode();
+    NodeStatus status = NodeStatus.inServiceHealthy();
+    map.addNode(dn, status);
+    assertEquals(dn, map.getNodeInfo(dn.getUuid()));
+    assertEquals(status, map.getNodeStatus(dn.getUuid()));
+  }
+
+  @Test
+  public void testNodeHealthStateCanBeUpdated()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    DatanodeDetails dn = generateDatanode();
+    NodeStatus status = NodeStatus.inServiceHealthy();
+    map.addNode(dn, status);
+
+    NodeStatus expectedStatus = NodeStatus.inServiceStale();
+    NodeStatus returnedStatus =
+        map.updateNodeHealthState(dn.getUuid(), expectedStatus.getHealth());
+    assertEquals(expectedStatus, returnedStatus);
+    assertEquals(returnedStatus, map.getNodeStatus(dn.getUuid()));
+  }
+
+  @Test
+  public void testNodeOperationalStateCanBeUpdated()
+      throws NodeAlreadyExistsException, NodeNotFoundException {
+    DatanodeDetails dn = generateDatanode();
+    NodeStatus status = NodeStatus.inServiceHealthy();
+    map.addNode(dn, status);
+
+    NodeStatus expectedStatus = new NodeStatus(
+        NodeOperationalState.DECOMMISSIONING,
+        NodeState.HEALTHY, 999);
+    NodeStatus returnedStatus = map.updateNodeOperationalState(
+        dn.getUuid(), expectedStatus.getOperationalState(), 999);
+    assertEquals(expectedStatus, returnedStatus);
+    assertEquals(returnedStatus, map.getNodeStatus(dn.getUuid()));
+    assertEquals(999, returnedStatus.getOpStateExpiryEpochSeconds());
+  }
+
+  @Test
+  public void testGetNodeMethodsReturnCorrectCountsAndStates()
+      throws NodeAlreadyExistsException {
+    // Add one node for all possible states
+    int nodeCount = 0;
+    for(NodeOperationalState op : NodeOperationalState.values()) {
+      for(NodeState health : NodeState.values()) {
+        addRandomNodeWithState(op, health);
+        nodeCount++;
+      }
+    }
+    NodeStatus requestedState = NodeStatus.inServiceStale();
+    List<UUID> nodes = map.getNodes(requestedState);
+    assertEquals(1, nodes.size());
+    assertEquals(1, map.getNodeCount(requestedState));
+    assertEquals(nodeCount, map.getTotalNodeCount());
+    assertEquals(nodeCount, map.getAllNodes().size());
+    assertEquals(nodeCount, map.getAllDatanodeInfos().size());
+
+    // Checks for the getNodeCount(opstate, health) method
+    assertEquals(nodeCount, map.getNodeCount(null, null));
+    assertEquals(1,
+        map.getNodeCount(NodeOperationalState.DECOMMISSIONING,
+            NodeState.STALE));
+    assertEquals(5, map.getNodeCount(null, NodeState.HEALTHY));
+    assertEquals(3,
+        map.getNodeCount(NodeOperationalState.DECOMMISSIONING, null));
+  }
+
+  private void addNodeWithState(DatanodeDetails dn,
+      NodeOperationalState opState, NodeState health)
+      throws NodeAlreadyExistsException {
+    NodeStatus status = new NodeStatus(opState, health);
+    map.addNode(dn, status);
+  }
+
+  private void addRandomNodeWithState(
+      NodeOperationalState opState, NodeState health)
+      throws NodeAlreadyExistsException {
+    DatanodeDetails dn = generateDatanode();
+    addNodeWithState(dn, opState, health);
+  }
+
+  private DatanodeDetails generateDatanode() {
+    return DatanodeDetails.newBuilder().setUuid(UUID.randomUUID()).build();
+  }
+
+}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
index 41eea3d..3f2ed2c 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelineDatanodesIntersection.java
@@ -23,6 +23,7 @@
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -85,7 +86,7 @@
         stateManager, conf);
 
     int healthyNodeCount = nodeManager
-        .getNodeCount(HddsProtos.NodeState.HEALTHY);
+        .getNodeCount(NodeStatus.inServiceHealthy());
     int intersectionCount = 0;
     int createdPipelineCount = 0;
     while (!end && createdPipelineCount <= healthyNodeCount * nodeHeaviness) {
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
index f024fc5..8a2b9c9 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestPipelinePlacementPolicy.java
@@ -33,6 +33,7 @@
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.net.NetConstants;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.net.NetworkTopologyImpl;
@@ -175,7 +176,7 @@
   @Test
   public void testPickLowestLoadAnchor() throws IOException{
     List<DatanodeDetails> healthyNodes = nodeManager
-        .getNodes(HddsProtos.NodeState.HEALTHY);
+        .getNodes(NodeStatus.inServiceHealthy());
 
     int maxPipelineCount = PIPELINE_LOAD_LIMIT * healthyNodes.size()
         / HddsProtos.ReplicationFactor.THREE.getNumber();
@@ -215,7 +216,7 @@
   @Test
   public void testChooseNodeBasedOnRackAwareness() {
     List<DatanodeDetails> healthyNodes = overWriteLocationInNodes(
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY));
+        nodeManager.getNodes(NodeStatus.inServiceHealthy()));
     DatanodeDetails anchor = placementPolicy.chooseNode(healthyNodes);
     NetworkTopology topologyWithDifRacks =
         createNetworkTopologyOnDifRacks();
@@ -231,7 +232,7 @@
   @Test
   public void testFallBackPickNodes() {
     List<DatanodeDetails> healthyNodes = overWriteLocationInNodes(
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY));
+        nodeManager.getNodes(NodeStatus.inServiceHealthy()));
     DatanodeDetails node;
     try {
       node = placementPolicy.fallBackPickNodes(healthyNodes, null);
@@ -338,7 +339,7 @@
   @Test
   public void testHeavyNodeShouldBeExcluded() throws SCMException{
     List<DatanodeDetails> healthyNodes =
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY);
+        nodeManager.getNodes(NodeStatus.inServiceHealthy());
     int nodesRequired = HddsProtos.ReplicationFactor.THREE.getNumber();
     // only minority of healthy NODES are heavily engaged in pipelines.
     int minorityHeavy = healthyNodes.size()/2 - 1;
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
index 9173302..383944d 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineProvider.java
@@ -25,6 +25,7 @@
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.hdds.scm.container.MockNodeManager;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Test;
@@ -146,7 +147,7 @@
       throws Exception {
     init(2);
     List<DatanodeDetails> healthyNodes = nodeManager
-        .getNodes(HddsProtos.NodeState.HEALTHY).stream()
+        .getNodes(NodeStatus.inServiceHealthy()).stream()
         .limit(3).collect(Collectors.toList());
 
     Pipeline pipeline1 = provider.create(
@@ -163,7 +164,7 @@
     int maxPipelinePerNode = 2;
     init(maxPipelinePerNode);
     List<DatanodeDetails> healthyNodes =
-        nodeManager.getNodes(HddsProtos.NodeState.HEALTHY);
+        nodeManager.getNodes(NodeStatus.inServiceHealthy());
 
     Assume.assumeTrue(healthyNodes.size() == 8);
 
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
index d14e468..1ea8dbf 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
@@ -45,6 +45,7 @@
 import org.apache.hadoop.hdds.scm.metadata.PipelineIDCodec;
 import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
 import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
 import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
 import org.apache.hadoop.hdds.server.events.EventQueue;
@@ -786,7 +787,7 @@
         .setState(Pipeline.PipelineState.OPEN)
         .setNodes(
             Arrays.asList(
-                nodeManager.getNodes(HddsProtos.NodeState.HEALTHY).get(0)
+                nodeManager.getNodes(NodeStatus.inServiceHealthy()).get(0)
             )
         )
         .setNodesInOrder(Arrays.asList(0))
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
index a0cf957..1656376 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/placement/TestContainerPlacement.java
@@ -27,10 +27,9 @@
 import org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.ozone.OzoneConsts;
-
 import org.apache.commons.math3.stat.descriptive.DescriptiveStatistics;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
 import org.junit.Assert;
 import static org.junit.Assert.assertEquals;
 import org.junit.Test;
@@ -42,7 +41,8 @@
 
   private DescriptiveStatistics computeStatistics(NodeManager nodeManager) {
     DescriptiveStatistics descriptiveStatistics = new DescriptiveStatistics();
-    for (DatanodeDetails dd : nodeManager.getNodes(HEALTHY)) {
+    for (DatanodeDetails dd :
+        nodeManager.getNodes(NodeStatus.inServiceHealthy())) {
       float weightedValue =
           nodeManager.getNodeStat(dd).get().getScmUsed().get() / (float)
               nodeManager.getNodeStat(dd).get().getCapacity().get();
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
index 6d088fe..9d428c2 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/container/testutils/ReplicationNodeManagerMock.java
@@ -17,9 +17,11 @@
 package org.apache.hadoop.ozone.container.testutils;
 
 import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.protocol.proto
         .StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
 import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.net.NetworkTopology;
 import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
 import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
@@ -51,17 +53,17 @@
  * A Node Manager to test replication.
  */
 public class ReplicationNodeManagerMock implements NodeManager {
-  private final Map<DatanodeDetails, NodeState> nodeStateMap;
+  private final Map<DatanodeDetails, NodeStatus> nodeStateMap;
   private final CommandQueue commandQueue;
 
   /**
    * A list of Datanodes and current states.
-   * @param nodeState A node state map.
+   * @param nodeStatus A node state map.
    */
-  public ReplicationNodeManagerMock(Map<DatanodeDetails, NodeState> nodeState,
+  public ReplicationNodeManagerMock(Map<DatanodeDetails, NodeStatus> nodeStatus,
                                     CommandQueue commandQueue) {
-    Preconditions.checkNotNull(nodeState);
-    this.nodeStateMap = nodeState;
+    Preconditions.checkNotNull(nodeStatus);
+    this.nodeStateMap = nodeStatus;
     this.commandQueue = commandQueue;
   }
 
@@ -71,7 +73,7 @@
    * @return A state to number of nodes that in this state mapping
    */
   @Override
-  public Map<String, Integer> getNodeCount() {
+  public Map<String, Map<String, Integer>> getNodeCount() {
     return null;
   }
 
@@ -83,22 +85,48 @@
   /**
    * Gets all Live Datanodes that is currently communicating with SCM.
    *
-   * @param nodestate - State of the node
+   * @param nodestatus - State of the node
    * @return List of Datanodes that are Heartbeating SCM.
    */
   @Override
-  public List<DatanodeDetails> getNodes(NodeState nodestate) {
+  public List<DatanodeDetails> getNodes(NodeStatus nodestatus) {
+    return null;
+  }
+
+  /**
+   * Gets all Live Datanodes that is currently communicating with SCM.
+   *
+   * @param opState - Operational state of the node
+   * @param health - Health of the node
+   * @return List of Datanodes that are Heartbeating SCM.
+   */
+  @Override
+  public List<DatanodeDetails> getNodes(
+      HddsProtos.NodeOperationalState opState, NodeState health) {
     return null;
   }
 
   /**
    * Returns the Number of Datanodes that are communicating with SCM.
    *
-   * @param nodestate - State of the node
+   * @param nodestatus - State of the node
    * @return int -- count
    */
   @Override
-  public int getNodeCount(NodeState nodestate) {
+  public int getNodeCount(NodeStatus nodestatus) {
+    return 0;
+  }
+
+  /**
+   * Returns the Number of Datanodes that are communicating with SCM.
+   *
+   * @param opState - Operational state of the node
+   * @param health - Health of the node
+   * @return int -- count
+   */
+  @Override
+  public int getNodeCount(
+      HddsProtos.NodeOperationalState opState, NodeState health) {
     return 0;
   }
 
@@ -152,11 +180,40 @@
    * @return Healthy/Stale/Dead.
    */
   @Override
-  public NodeState getNodeState(DatanodeDetails dd) {
+  public NodeStatus getNodeStatus(DatanodeDetails dd) {
     return nodeStateMap.get(dd);
   }
 
   /**
+   * Set the operation state of a node.
+   * @param dd The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  @Override
+  public void setNodeOperationalState(DatanodeDetails dd,
+      HddsProtos.NodeOperationalState newState) throws NodeNotFoundException {
+    setNodeOperationalState(dd, newState, 0);
+  }
+
+  /**
+   * Set the operation state of a node.
+   * @param dd The datanode to set the new state for
+   * @param newState The new operational state for the node
+   */
+  @Override
+  public void setNodeOperationalState(DatanodeDetails dd,
+      HddsProtos.NodeOperationalState newState, long opStateExpiryEpocSec)
+      throws NodeNotFoundException {
+    NodeStatus currentStatus = nodeStateMap.get(dd);
+    if (currentStatus != null) {
+      nodeStateMap.put(dd, new NodeStatus(newState, currentStatus.getHealth(),
+          opStateExpiryEpocSec));
+    } else {
+      throw new NodeNotFoundException();
+    }
+  }
+
+  /**
    * Get set of pipelines a datanode is part of.
    * @param dnId - datanodeID
    * @return Set of PipelineID
@@ -296,10 +353,10 @@
    * Adds a node to the existing Node manager. This is used only for test
    * purposes.
    * @param id DatanodeDetails
-   * @param state State you want to put that node to.
+   * @param status State you want to put that node to.
    */
-  public void addNode(DatanodeDetails id, NodeState state) {
-    nodeStateMap.put(id, state);
+  public void addNode(DatanodeDetails id, NodeStatus status) {
+    nodeStateMap.put(id, status);
   }
 
   @Override
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java
index 7576e8b..fd65283 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/ozone/scm/node/TestSCMNodeMetrics.java
@@ -172,17 +172,72 @@
 
     MetricsRecordBuilder metricsSource = getMetrics(SCMNodeMetrics.SOURCE_NAME);
 
-    assertGauge("HealthyNodes", 1, metricsSource);
-    assertGauge("StaleNodes", 0, metricsSource);
-    assertGauge("DeadNodes", 0, metricsSource);
-    assertGauge("DecommissioningNodes", 0, metricsSource);
-    assertGauge("DecommissionedNodes", 0, metricsSource);
-    assertGauge("DiskCapacity", 100L, metricsSource);
-    assertGauge("DiskUsed", 10L, metricsSource);
-    assertGauge("DiskRemaining", 90L, metricsSource);
-    assertGauge("SSDCapacity", 0L, metricsSource);
-    assertGauge("SSDUsed", 0L, metricsSource);
-    assertGauge("SSDRemaining", 0L, metricsSource);
+    assertGauge("InServiceHealthyNodes", 1,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("InServiceStaleNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("InServiceDeadNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissioningHealthyNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissioningStaleNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissioningDeadNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedHealthyNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedStaleNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedDeadNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("EnteringMaintenanceHealthyNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("EnteringMaintenanceStaleNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("EnteringMaintenanceDeadNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("InMaintenanceHealthyNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("InMaintenanceStaleNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("InMaintenanceDeadNodes", 0,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DiskCapacity", 100L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DiskUsed", 10L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DiskRemaining", 90L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("SSDCapacity", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("SSDUsed", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("SSDRemaining", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceDiskCapacity", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceDiskUsed", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceDiskRemaining", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceSSDCapacity", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceSSDUsed", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("MaintenanceSSDRemaining", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedDiskCapacity", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedDiskUsed", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedDiskRemaining", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedSSDCapacity", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedSSDUsed", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
+    assertGauge("DecommissionedSSDRemaining", 0L,
+        getMetrics(SCMNodeMetrics.class.getSimpleName()));
 
   }
 
diff --git a/hadoop-hdds/tools/pom.xml b/hadoop-hdds/tools/pom.xml
index dfda5a6..9da6f93 100644
--- a/hadoop-hdds/tools/pom.xml
+++ b/hadoop-hdds/tools/pom.xml
@@ -78,6 +78,12 @@
       <groupId>org.xerial</groupId>
       <artifactId>sqlite-jdbc</artifactId>
     </dependency>
+      <dependency>
+          <groupId>org.mockito</groupId>
+          <artifactId>mockito-all</artifactId>
+          <version>${mockito1-hadoop.version}</version>
+          <scope>test</scope>
+      </dependency>
 
   </dependencies>
 </project>
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
index 96cd530..2f4e66e 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java
@@ -259,18 +259,39 @@
   /**
    * Returns a set of Nodes that meet a query criteria.
    *
-   * @param nodeStatuses - Criteria that we want the node to have.
-   * @param queryScope   - Query scope - Cluster or pool.
-   * @param poolName     - if it is pool, a pool name is required.
+   * @param opState - The operational state we want the node to have
+   *                eg IN_SERVICE, DECOMMISSIONED, etc
+   * @param nodeState - The health we want the node to have, eg HEALTHY, STALE,
+   *                  etc
+   * @param queryScope - Query scope - Cluster or pool.
+   * @param poolName - if it is pool, a pool name is required.
    * @return A set of nodes that meet the requested criteria.
    * @throws IOException
    */
   @Override
-  public List<HddsProtos.Node> queryNode(HddsProtos.NodeState
-      nodeStatuses, HddsProtos.QueryScope queryScope, String poolName)
+  public List<HddsProtos.Node> queryNode(
+      HddsProtos.NodeOperationalState opState,
+      HddsProtos.NodeState nodeState,
+      HddsProtos.QueryScope queryScope, String poolName)
       throws IOException {
-    return storageContainerLocationClient.queryNode(nodeStatuses, queryScope,
-        poolName);
+    return storageContainerLocationClient.queryNode(opState, nodeState,
+        queryScope, poolName);
+  }
+
+  @Override
+  public void decommissionNodes(List<String> hosts) throws IOException {
+    storageContainerLocationClient.decommissionNodes(hosts);
+  }
+
+  @Override
+  public void recommissionNodes(List<String> hosts) throws IOException {
+    storageContainerLocationClient.recommissionNodes(hosts);
+  }
+
+  @Override
+  public void startMaintenanceNodes(List<String> hosts, int endHours)
+      throws IOException {
+    storageContainerLocationClient.startMaintenanceNodes(hosts, endHours);
   }
 
   /**
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
index c1aebae..bb442e0 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/TopologySubcommand.java
@@ -34,8 +34,6 @@
 import org.apache.hadoop.hdds.scm.client.ScmClient;
 
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DECOMMISSIONED;
-import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DECOMMISSIONING;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
 
@@ -60,8 +58,6 @@
     STATES.add(HEALTHY);
     STATES.add(STALE);
     STATES.add(DEAD);
-    STATES.add(DECOMMISSIONING);
-    STATES.add(DECOMMISSIONED);
   }
 
   @CommandLine.Option(names = {"-o", "--order"},
@@ -73,16 +69,11 @@
   private boolean fullInfo;
 
   @Override
-  public Class<?> getParentType() {
-    return OzoneAdmin.class;
-  }
-
-  @Override
-  protected void execute(ScmClient scmClient) throws IOException {
+  public void execute(ScmClient scmClient) throws IOException {
     for (HddsProtos.NodeState state : STATES) {
-      List<HddsProtos.Node> nodes = scmClient.queryNode(state,
+      List<HddsProtos.Node> nodes = scmClient.queryNode(null, state,
           HddsProtos.QueryScope.CLUSTER, "");
-      if (nodes != null && !nodes.isEmpty()) {
+      if (nodes != null && nodes.size() > 0) {
         // show node state
         System.out.println("State = " + state.toString());
         if (order) {
@@ -94,27 +85,36 @@
     }
   }
 
+  public Class<?> getParentType() {
+    return OzoneAdmin.class;
+  }
+
   // Format
   // Location: rack1
-  //  ipAddress(hostName)
+  //  ipAddress(hostName) OperationalState
   private void printOrderedByLocation(List<HddsProtos.Node> nodes) {
     HashMap<String, TreeSet<DatanodeDetails>> tree =
         new HashMap<>();
+    HashMap<DatanodeDetails, HddsProtos.NodeOperationalState> state =
+        new HashMap<>();
+
     for (HddsProtos.Node node : nodes) {
       String location = node.getNodeID().getNetworkLocation();
       if (location != null && !tree.containsKey(location)) {
         tree.put(location, new TreeSet<>());
       }
-      tree.get(location).add(DatanodeDetails.getFromProtoBuf(node.getNodeID()));
+      DatanodeDetails dn = DatanodeDetails.getFromProtoBuf(node.getNodeID());
+      tree.get(location).add(dn);
+      state.put(dn, node.getNodeOperationalStates(0));
     }
     ArrayList<String> locations = new ArrayList<>(tree.keySet());
     Collections.sort(locations);
 
     locations.forEach(location -> {
       System.out.println("Location: " + location);
-      tree.get(location).forEach(node -> {
-        System.out.println(" " + node.getIpAddress() + "(" + node.getHostName()
-            + ")");
+      tree.get(location).forEach(n -> {
+        System.out.println(" " + n.getIpAddress() + "(" + n.getHostName()
+            + ") "+state.get(n));
       });
     });
   }
@@ -135,16 +135,18 @@
     return fullInfo ? node.getNodeID().getUuid() + "/" : "";
   }
 
-  // Format "ipAddress(hostName):PortName1=PortValue1    networkLocation"
+  // Format "ipAddress(hostName):PortName1=PortValue1    OperationalState
+  //     networkLocation
   private void printNodesWithLocation(Collection<HddsProtos.Node> nodes) {
     nodes.forEach(node -> {
       System.out.print(" " + getAdditionNodeOutput(node) +
           node.getNodeID().getIpAddress() + "(" +
           node.getNodeID().getHostName() + ")" +
           ":" + formatPortOutput(node.getNodeID().getPortsList()));
-      System.out.println("    " +
+      System.out.println("    "
+          + node.getNodeOperationalStates(0) + "    " +
           (node.getNodeID().getNetworkLocation() != null ?
               node.getNodeID().getNetworkLocation() : "NA"));
     });
   }
-}
+}
\ No newline at end of file
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
index 7e77c60..4f8d4d1 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DatanodeCommands.java
@@ -38,7 +38,10 @@
     mixinStandardHelpOptions = true,
     versionProvider = HddsVersionProvider.class,
     subcommands = {
-        ListInfoSubcommand.class
+        ListInfoSubcommand.class,
+        DecommissionSubCommand.class,
+        MaintenanceSubCommand.class,
+        RecommissionSubCommand.class
     })
 @MetaInfServices(SubcommandWithParent.class)
 public class DatanodeCommands implements Callable<Void>, SubcommandWithParent {
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DecommissionSubCommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DecommissionSubCommand.java
new file mode 100644
index 0000000..a4e7e3f
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/DecommissionSubCommand.java
@@ -0,0 +1,46 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.datanode;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Decommission one or more datanodes.
+ */
+@Command(
+    name = "decommission",
+    description = "Decommission a datanode",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class DecommissionSubCommand extends ScmSubcommand {
+
+  @CommandLine.Parameters(description = "List of fully qualified host names")
+  private List<String> hosts = new ArrayList<String>();
+
+  @Override
+  public void execute(ScmClient scmClient) throws IOException {
+    scmClient.decommissionNodes(hosts);
+  }
+}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
index 80c5eca..38ad390 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/ListInfoSubcommand.java
@@ -60,30 +60,34 @@
     if (Strings.isNullOrEmpty(ipaddress) && Strings.isNullOrEmpty(uuid)) {
       getAllNodes(scmClient).forEach(this::printDatanodeInfo);
     } else {
-      Stream<DatanodeDetails> allNodes = getAllNodes(scmClient).stream();
+      Stream<DatanodeWithAttributes> allNodes = getAllNodes(scmClient).stream();
       if (!Strings.isNullOrEmpty(ipaddress)) {
-        allNodes = allNodes.filter(p -> p.getIpAddress()
+        allNodes = allNodes.filter(p -> p.getDatanodeDetails().getIpAddress()
             .compareToIgnoreCase(ipaddress) == 0);
       }
       if (!Strings.isNullOrEmpty(uuid)) {
-        allNodes = allNodes.filter(p -> p.getUuid().toString().equals(uuid));
+        allNodes = allNodes.filter(p ->
+            p.getDatanodeDetails().toString().equals(uuid));
       }
       allNodes.forEach(this::printDatanodeInfo);
     }
   }
 
-  private List<DatanodeDetails> getAllNodes(ScmClient scmClient)
+  private List<DatanodeWithAttributes> getAllNodes(ScmClient scmClient)
       throws IOException {
-    List<HddsProtos.Node> nodes = scmClient.queryNode(
+    List<HddsProtos.Node> nodes = scmClient.queryNode(null,
         HddsProtos.NodeState.HEALTHY, HddsProtos.QueryScope.CLUSTER, "");
 
     return nodes.stream()
-        .map(p -> DatanodeDetails.getFromProtoBuf(p.getNodeID()))
+        .map(p -> new DatanodeWithAttributes(
+            DatanodeDetails.getFromProtoBuf(p.getNodeID()),
+            p.getNodeOperationalStates(0), p.getNodeStates(0)))
         .collect(Collectors.toList());
   }
 
-  private void printDatanodeInfo(DatanodeDetails datanode) {
+  private void printDatanodeInfo(DatanodeWithAttributes dna) {
     StringBuilder pipelineListInfo = new StringBuilder();
+    DatanodeDetails datanode = dna.getDatanodeDetails();
     int relatedPipelineNum = 0;
     if (!pipelines.isEmpty()) {
       List<Pipeline> relatedPipelines = pipelines.stream().filter(
@@ -108,6 +112,34 @@
     System.out.println("Datanode: " + datanode.getUuid().toString() +
         " (" + datanode.getNetworkLocation() + "/" + datanode.getIpAddress()
         + "/" + datanode.getHostName() + "/" + relatedPipelineNum +
-        " pipelines) \n" + "Related pipelines: \n" + pipelineListInfo);
+        " pipelines)");
+    System.out.println("Operational State: " + dna.getOpState());
+    System.out.println("Related pipelines: \n" + pipelineListInfo);
+  }
+
+  private static class DatanodeWithAttributes {
+    private DatanodeDetails datanodeDetails;
+    private HddsProtos.NodeOperationalState operationalState;
+    private HddsProtos.NodeState healthState;
+
+    DatanodeWithAttributes(DatanodeDetails dn,
+        HddsProtos.NodeOperationalState opState,
+        HddsProtos.NodeState healthState) {
+      this.datanodeDetails = dn;
+      this.operationalState = opState;
+      this.healthState = healthState;
+    }
+
+    public DatanodeDetails getDatanodeDetails() {
+      return datanodeDetails;
+    }
+
+    public HddsProtos.NodeOperationalState getOpState() {
+      return operationalState;
+    }
+
+    public HddsProtos.NodeState getHealthState() {
+      return healthState;
+    }
   }
 }
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/MaintenanceSubCommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/MaintenanceSubCommand.java
new file mode 100644
index 0000000..fa1d802
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/MaintenanceSubCommand.java
@@ -0,0 +1,51 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.datanode;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Place one or more datanodes into Maintenance Mode.
+ */
+@Command(
+    name = "maintenance",
+    description = "Put a datanode into Maintenance Mode",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class MaintenanceSubCommand extends ScmSubcommand {
+
+  @CommandLine.Parameters(description = "List of fully qualified host names")
+  private List<String> hosts = new ArrayList<String>();
+
+  @CommandLine.Option(names = {"--end"},
+      description = "Automatically end maintenance after the given hours. "+
+          "By default, maintenance must be ended manually.")
+  private int endInHours = 0;
+
+  @Override
+  public void execute(ScmClient scmClient) throws IOException {
+    scmClient.startMaintenanceNodes(hosts, endInHours);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/RecommissionSubCommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/RecommissionSubCommand.java
new file mode 100644
index 0000000..b6e2f3d
--- /dev/null
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/datanode/RecommissionSubCommand.java
@@ -0,0 +1,46 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.cli.datanode;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.scm.cli.ScmSubcommand;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Place decommissioned or maintenance nodes back into service.
+ */
+@Command(
+    name = "recommission",
+    description = "Return a datanode to service",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class RecommissionSubCommand extends ScmSubcommand {
+
+  @CommandLine.Parameters(description = "List of fully qualified host names")
+  private List<String> hosts = new ArrayList<String>();
+
+  @Override
+  public void execute(ScmClient scmClient) throws IOException {
+    scmClient.recommissionNodes(hosts);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java
new file mode 100644
index 0000000..45d4d7b
--- /dev/null
+++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestListInfoSubcommand.java
@@ -0,0 +1,119 @@
+/*
+ * 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.cli.datanode;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.client.ScmClient;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.any;
+
+/**
+ * Unit tests to validate the the TestListInfoSubCommand class includes the
+ * correct output when executed against a mock client.
+ */
+public class TestListInfoSubcommand {
+
+  private ListInfoSubcommand cmd;
+  private final ByteArrayOutputStream outContent = new ByteArrayOutputStream();
+  private final ByteArrayOutputStream errContent = new ByteArrayOutputStream();
+  private final PrintStream originalOut = System.out;
+  private final PrintStream originalErr = System.err;
+
+  @Before
+  public void setup() {
+    cmd = new ListInfoSubcommand();
+    System.setOut(new PrintStream(outContent));
+    System.setErr(new PrintStream(errContent));
+  }
+
+  @After
+  public void tearDown() {
+    System.setOut(originalOut);
+    System.setErr(originalErr);
+  }
+
+  @Test
+  public void testDataNodeOperationalStateIncludedInOutput() throws Exception {
+    ScmClient scmClient = mock(ScmClient.class);
+    Mockito.when(scmClient.queryNode(any(HddsProtos.NodeOperationalState.class),
+        any(HddsProtos.NodeState.class), any(HddsProtos.QueryScope.class),
+        Mockito.anyString()))
+        .thenAnswer(invocation -> getNodeDetails());
+    Mockito.when(scmClient.listPipelines())
+        .thenReturn(new ArrayList<>());
+
+    cmd.execute(scmClient);
+
+    // The output should contain a string like:
+    // <other lines>
+    // Operational State: <STATE>
+    // <other lines>
+    Pattern p = Pattern.compile(
+        "^Operational State:\\s+IN_SERVICE$", Pattern.MULTILINE);
+    Matcher m = p.matcher(outContent.toString());
+    assertTrue(m.find());
+    // Should also have a node with the state DECOMMISSIONING
+    p = Pattern.compile(
+        "^Operational State:\\s+DECOMMISSIONING$", Pattern.MULTILINE);
+    m = p.matcher(outContent.toString());
+    assertTrue(m.find());
+  }
+
+  private List<HddsProtos.Node> getNodeDetails() {
+    List<HddsProtos.Node> nodes = new ArrayList<>();
+
+    for (int i=0; i<2; i++) {
+      HddsProtos.DatanodeDetailsProto.Builder dnd =
+          HddsProtos.DatanodeDetailsProto.newBuilder();
+      dnd.setHostName("host" + i);
+      dnd.setIpAddress("1.2.3." + i+1);
+      dnd.setNetworkLocation("/default");
+      dnd.setNetworkName("host" + i);
+      dnd.addPorts(HddsProtos.Port.newBuilder()
+          .setName("ratis").setValue(5678).build());
+      dnd.setUuid(UUID.randomUUID().toString());
+
+      HddsProtos.Node.Builder builder  = HddsProtos.Node.newBuilder();
+      if (i == 0) {
+        builder.addNodeOperationalStates(
+            HddsProtos.NodeOperationalState.IN_SERVICE);
+      } else {
+        builder.addNodeOperationalStates(
+            HddsProtos.NodeOperationalState.DECOMMISSIONING);
+      }
+      builder.addNodeStates(HddsProtos.NodeState.HEALTHY);
+      builder.setNodeID(dnd.build());
+      nodes.add(builder.build());
+    }
+    return nodes;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/dist/src/main/smoketest/topology/cli.robot b/hadoop-ozone/dist/src/main/smoketest/topology/cli.robot
index 3f83ba3..bbe7a1b 100644
--- a/hadoop-ozone/dist/src/main/smoketest/topology/cli.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/topology/cli.robot
@@ -26,8 +26,8 @@
 *** Test Cases ***
 Run printTopology
     ${output} =         Execute          ozone admin printTopology
-                        Should contain   ${output}         10.5.0.7(ozone-topology_datanode_4_1.ozone-topology_net)    /rack2
+                        Should contain   ${output}         10.5.0.7(ozone-topology_datanode_4_1.ozone-topology_net)    IN_SERVICE    /rack2
 Run printTopology -o
     ${output} =         Execute          ozone admin printTopology -o
                         Should contain   ${output}         Location: /rack2
-                        Should contain   ${output}         10.5.0.7(ozone-topology_datanode_4_1.ozone-topology_net)
+                        Should contain   ${output}         10.5.0.7(ozone-topology_datanode_4_1.ozone-topology_net) IN_SERVICE
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineCreateAndDestroy.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineCreateAndDestroy.java
index 6236900..b16add0 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineCreateAndDestroy.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestRatisPipelineCreateAndDestroy.java
@@ -21,6 +21,7 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.ozone.HddsDatanodeService;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
@@ -156,7 +157,7 @@
     }
 
     if (cluster.getStorageContainerManager()
-        .getScmNodeManager().getNodeCount(HddsProtos.NodeState.HEALTHY) >=
+        .getScmNodeManager().getNodeCount(NodeStatus.inServiceHealthy()) >=
         HddsProtos.ReplicationFactor.THREE.getNumber()) {
       // make sure pipelines is created after node start
       pipelineManager.triggerPipelineCreation();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
index 2b492a2..00163a6 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java
@@ -53,7 +53,6 @@
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.DeletedBlocksTransaction;
 import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
@@ -70,6 +69,7 @@
 import org.apache.hadoop.hdds.scm.exceptions.SCMException;
 import org.apache.hadoop.hdds.scm.node.DatanodeInfo;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
 import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@@ -373,8 +373,7 @@
         NodeManager nodeManager = cluster.getStorageContainerManager()
             .getScmNodeManager();
         List<SCMCommand> commands = nodeManager.processHeartbeat(
-            nodeManager.getNodes(NodeState.HEALTHY).get(0));
-
+            nodeManager.getNodes(NodeStatus.inServiceHealthy()).get(0));
         if (commands != null) {
           for (SCMCommand cmd : commands) {
             if (cmd.getType() == SCMCommandProto.Type.deleteBlocksCommand) {
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMNodeManagerMXBean.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMNodeManagerMXBean.java
index 5eba2f5..fd4d3db 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMNodeManagerMXBean.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMNodeManagerMXBean.java
@@ -100,10 +100,32 @@
             + "name=SCMNodeManagerInfo");
 
     TabularData data = (TabularData) mbs.getAttribute(bean, "NodeCount");
-    Map<String, Integer> nodeCount = scm.getScmNodeManager().getNodeCount();
-    Map<String, Long> nodeCountLong = new HashMap<>();
-    nodeCount.forEach((k, v) -> nodeCountLong.put(k, new Long(v)));
-    verifyEquals(data, nodeCountLong);
+    Map<String, Map<String, Integer>> mbeanMap = convertNodeCountToMap(data);
+    Map<String, Map<String, Integer>> nodeMap =
+        scm.getScmNodeManager().getNodeCount();
+    assertTrue(nodeMap.equals(mbeanMap));
+  }
+
+  private Map<String, Map<String, Integer>> convertNodeCountToMap(
+      TabularData data) {
+    Map<String, Map<String, Integer>> map = new HashMap<>();
+    for (Object o : data.values()) {
+      CompositeData cds = (CompositeData) o;
+      Iterator<?> it = cds.values().iterator();
+      String opState = it.next().toString();
+      TabularData states = (TabularData) it.next();
+
+      Map<String, Integer> healthStates = new HashMap<>();
+      for (Object obj : states.values()) {
+        CompositeData stateData = (CompositeData) obj;
+        Iterator<?> stateIt = stateData.values().iterator();
+        String health = stateIt.next().toString();
+        Integer value = Integer.parseInt(stateIt.next().toString());
+        healthStates.put(health, value);
+      }
+      map.put(opState, healthStates);
+    }
+    return map;
   }
 
   private void verifyEquals(TabularData actualData, Map<String, Long>
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestDecommissionAndMaintenance.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestDecommissionAndMaintenance.java
new file mode 100644
index 0000000..42b4cd8
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestDecommissionAndMaintenance.java
@@ -0,0 +1,695 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.scm.node;
+
+import org.apache.hadoop.hdds.client.ReplicationFactor;
+import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient;
+import org.apache.hadoop.hdds.scm.container.ContainerInfo;
+import org.apache.hadoop.hdds.scm.container.ContainerManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.container.ContainerReplica;
+import org.apache.hadoop.hdds.scm.container.ContainerReplicaCount;
+import org.apache.hadoop.hdds.scm.container.ReplicationManager.ReplicationManagerConfiguration;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
+import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.assertTrue;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_COMMAND_STATUS_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_CONTAINER_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONED;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_MAINTENANCE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL;
+import static org.junit.Assert.fail;
+
+/**
+ * Test from the scmclient for decommission and maintenance.
+ */
+
+public class TestDecommissionAndMaintenance {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestDecommissionAndMaintenance.class);
+
+  private static int numOfDatanodes = 6;
+  private static String bucketName = "bucket1";
+  private static String volName = "vol1";
+  private OzoneBucket bucket;
+  private MiniOzoneCluster cluster;
+  private NodeManager nm;
+  private ContainerManager cm;
+  private PipelineManager pm;
+  private StorageContainerManager scm;
+
+  private ContainerOperationClient scmClient;
+
+  @Before
+  public void setUp() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    final int interval = 100;
+
+    conf.setTimeDuration(OZONE_SCM_HEARTBEAT_PROCESS_INTERVAL,
+        interval, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_PIPELINE_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(HDDS_NODE_REPORT_INTERVAL, 1, SECONDS);
+    conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, SECONDS);
+    conf.setTimeDuration(OZONE_SCM_DEADNODE_INTERVAL, 6, SECONDS);
+    conf.setTimeDuration(OZONE_SCM_DATANODE_ADMIN_MONITOR_INTERVAL,
+        1, SECONDS);
+
+    ReplicationManagerConfiguration replicationConf =
+        conf.getObject(ReplicationManagerConfiguration.class);
+    replicationConf.setInterval(Duration.ofSeconds(1));
+    conf.setFromObject(replicationConf);
+
+    cluster = MiniOzoneCluster.newBuilder(conf)
+        .setNumDatanodes(numOfDatanodes)
+        .build();
+    cluster.waitForClusterToBeReady();
+    setManagers();
+
+    bucket = TestDataUtil.createVolumeAndBucket(cluster, volName, bucketName);
+    scmClient = new ContainerOperationClient(conf);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  // Decommissioning a node with open pipelines should close the pipelines
+  // and hence the open containers and then the containers should be replicated
+  // by the replication manager.
+  public void testNodeWithOpenPipelineCanBeDecommissioned()
+      throws Exception {
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+
+    // Locate any container and find its open pipeline
+    final ContainerInfo container = waitForAndReturnContainer();
+    Pipeline pipeline = pm.getPipeline(container.getPipelineID());
+    assertEquals(Pipeline.PipelineState.OPEN, pipeline.getPipelineState());
+    Set<ContainerReplica> replicas = getContainerReplicas(container);
+
+    final DatanodeDetails toDecommission = getOneDNHostingReplica(replicas);
+    scmClient.decommissionNodes(Arrays.asList(
+        getDNHostAndPort(toDecommission)));
+
+    // Ensure one node transitioned to DECOMMISSIONING
+    List<DatanodeDetails> decomNodes = nm.getNodes(
+        DECOMMISSIONING,
+        HEALTHY);
+    assertEquals(1, decomNodes.size());
+    waitForDnToReachOpState(toDecommission, DECOMMISSIONED);
+
+    // Should now be 4 replicas online as the DN is still alive but
+    // in the DECOMMISSIONED state.
+    Set<ContainerReplica> newReplicas =
+        cm.getContainerReplicas(container.containerID());
+    assertEquals(4, newReplicas.size());
+
+    // Stop the decommissioned DN
+    cluster.shutdownHddsDatanode(toDecommission);
+    waitForDnToReachHealthState(toDecommission, DEAD);
+
+    // Now the decommissioned node is dead, we should have
+    // 3 replicas for the tracked container.
+    newReplicas = cm.getContainerReplicas(container.containerID());
+    assertEquals(3, newReplicas.size());
+  }
+
+  @Test
+  // After a SCM restart, it will have forgotten all the Operational states.
+  // However the state will have been persisted on the DNs. Therefore on initial
+  // registration, the DN operationalState is the source of truth and SCM should
+  // be updated to reflect that.
+  public void testDecommissionedStateReinstatedAfterSCMRestart()
+      throws Exception {
+    // Decommission any node and wait for it to be DECOMMISSIONED
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    DatanodeDetails dn = nm.getAllNodes().get(0);
+    scmClient.decommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+    waitForDnToReachOpState(dn, DECOMMISSIONED);
+
+    cluster.restartStorageContainerManager(true);
+    setManagers();
+    DatanodeDetails newDn = nm.getNodeByUuid(dn.getUuid().toString());
+
+    // On initial registration, the DN should report its operational state
+    // and if it is decommissioned, that should be updated in the NodeStatus
+    waitForDnToReachOpState(newDn, DECOMMISSIONED);
+    // Also confirm the datanodeDetails correctly reflect the operational
+    // state.
+    waitForDnToReachPersistedOpState(newDn, DECOMMISSIONED);
+  }
+
+  @Test
+  // If a node has not yet completed decommission and SCM is restarted, then
+  // when it re-registers it should re-enter the decommission workflow and
+  // complete decommissioning.
+  public void testDecommissioningNodesCompleteDecommissionOnSCMRestart()
+      throws Exception {
+    // First stop the replicationManager so nodes marked for decommission cannot
+    // make any progress. THe node will be stuck DECOMMISSIONING
+    scm.getReplicationManager().stop();
+    // Generate some data and then pick a DN to decommission which is hosting a
+    // container. This ensures it will not decommission immediately due to
+    // having no containers.
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    final ContainerInfo container = waitForAndReturnContainer();
+    final DatanodeDetails dn
+        = getOneDNHostingReplica(getContainerReplicas(container));
+    scmClient.decommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+
+    // Wait for the state to be persisted on the DN so it can report it on
+    // restart of SCM.
+    waitForDnToReachPersistedOpState(dn, DECOMMISSIONING);
+    cluster.restartStorageContainerManager(true);
+    setManagers();
+
+    // After the SCM restart, the DN should report as DECOMMISSIONING, then
+    // it should re-enter the decommission workflow and move to DECOMMISSIONED
+    DatanodeDetails newDn = nm.getNodeByUuid(dn.getUuid().toString());
+    waitForDnToReachOpState(newDn, DECOMMISSIONED);
+    waitForDnToReachPersistedOpState(newDn, DECOMMISSIONED);
+  }
+
+  @Test
+  // If a node was decommissioned, and then stopped so it is dead. Then it is
+  // recommissioned in SCM and restarted, the SCM state should be taken as the
+  // source of truth and the node will go to the IN_SERVICE state and the state
+  // should be updated on the DN.
+  public void testStoppedDecommissionedNodeTakesSCMStateOnRestart()
+      throws Exception {
+    // Decommission node and wait for it to be DECOMMISSIONED
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+
+    DatanodeDetails dn = nm.getAllNodes().get(0);
+    scmClient.decommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+    waitForDnToReachOpState(dn, DECOMMISSIONED);
+    waitForDnToReachPersistedOpState(dn, DECOMMISSIONED);
+
+    int dnIndex = cluster.getHddsDatanodeIndex(dn);
+    cluster.shutdownHddsDatanode(dnIndex);
+    waitForDnToReachHealthState(dn, DEAD);
+
+    // Datanode is shutdown and dead. Now recommission it in SCM
+    scmClient.recommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+
+    // Now restart it and ensure it remains IN_SERVICE
+    cluster.restartHddsDatanode(dnIndex, true);
+    DatanodeDetails newDn = nm.getNodeByUuid(dn.getUuid().toString());
+
+    // As this is not an initial registration since SCM was started, the DN
+    // should report its operational state and if it differs from what SCM
+    // has, then the SCM state should be used and the DN state updated.
+    waitForDnToReachHealthState(newDn, HEALTHY);
+    waitForDnToReachOpState(newDn, IN_SERVICE);
+    waitForDnToReachPersistedOpState(dn, IN_SERVICE);
+  }
+
+  @Test
+  // A node which is decommissioning or decommissioned can be move back to
+  // IN_SERVICE.
+  public void testDecommissionedNodeCanBeRecommissioned() throws Exception {
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    DatanodeDetails dn = nm.getAllNodes().get(0);
+    scmClient.decommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+
+    GenericTestUtils.waitFor(
+        () -> !dn.getPersistedOpState()
+            .equals(IN_SERVICE),
+        200, 30000);
+
+    scmClient.recommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+    waitForDnToReachOpState(dn, IN_SERVICE);
+    waitForDnToReachPersistedOpState(dn, IN_SERVICE);
+  }
+
+  @Test
+  // When putting a single node into maintenance, its pipelines should be closed
+  // but no new replicas should be create and the node should transition into
+  // maintenance
+  public void testSingleNodeWithOpenPipelineCanGotoMaintenance()
+      throws Exception {
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+
+    // Locate any container and find its open pipeline
+    final ContainerInfo container = waitForAndReturnContainer();
+    Pipeline pipeline = pm.getPipeline(container.getPipelineID());
+    assertEquals(Pipeline.PipelineState.OPEN, pipeline.getPipelineState());
+    Set<ContainerReplica> replicas = getContainerReplicas(container);
+
+    final DatanodeDetails dn = getOneDNHostingReplica(replicas);
+    scmClient.startMaintenanceNodes(Arrays.asList(
+        getDNHostAndPort(dn)), 0);
+
+    waitForDnToReachOpState(dn, IN_MAINTENANCE);
+    waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+
+    // Should still be 3 replicas online as no replication should happen for
+    // maintenance
+    Set<ContainerReplica> newReplicas =
+        cm.getContainerReplicas(container.containerID());
+    assertEquals(3, newReplicas.size());
+
+    // Stop the maintenance DN
+    cluster.shutdownHddsDatanode(dn);
+    waitForDnToReachHealthState(dn, DEAD);
+
+    // Now the maintenance node is dead, we should still have
+    // 3 replicas as we don't purge the replicas for a dead maintenance node
+    newReplicas = cm.getContainerReplicas(container.containerID());
+    assertEquals(3, newReplicas.size());
+
+    // Restart the DN and it should keep the IN_MAINTENANCE state
+    cluster.restartHddsDatanode(dn, true);
+    DatanodeDetails newDN = nm.getNodeByUuid(dn.getUuid().toString());
+    waitForDnToReachHealthState(newDN, HEALTHY);
+    waitForDnToReachPersistedOpState(newDN, IN_MAINTENANCE);
+  }
+
+  @Test
+  // After a node enters maintenance and is stopped, it can be recommissioned in
+  // SCM. Then when it is restarted, it should go back to IN_SERVICE and have
+  // that persisted on the DN.
+  public void testStoppedMaintenanceNodeTakesScmStateOnRestart()
+      throws Exception {
+    // Put a node into maintenance and wait for it to complete
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    DatanodeDetails dn = nm.getAllNodes().get(0);
+    scmClient.startMaintenanceNodes(Arrays.asList(getDNHostAndPort(dn)), 0);
+    waitForDnToReachOpState(dn, IN_MAINTENANCE);
+    waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+
+    int dnIndex = cluster.getHddsDatanodeIndex(dn);
+    cluster.shutdownHddsDatanode(dnIndex);
+    waitForDnToReachHealthState(dn, DEAD);
+
+    // Datanode is shutdown and dead. Now recommission it in SCM
+    scmClient.recommissionNodes(Arrays.asList(getDNHostAndPort(dn)));
+
+    // Now restart it and ensure it remains IN_SERVICE
+    cluster.restartHddsDatanode(dnIndex, true);
+    DatanodeDetails newDn = nm.getNodeByUuid(dn.getUuid().toString());
+
+    // As this is not an initial registration since SCM was started, the DN
+    // should report its operational state and if it differs from what SCM
+    // has, then the SCM state should be used and the DN state updated.
+    waitForDnToReachHealthState(newDn, HEALTHY);
+    waitForDnToReachOpState(newDn, IN_SERVICE);
+    waitForDnToReachPersistedOpState(dn, IN_SERVICE);
+  }
+
+  @Test
+  // By default a node can enter maintenance if there are two replicas left
+  // available when the maintenance nodes are stopped. Therefore putting all
+  // nodes hosting a replica to maintenance should cause new replicas to get
+  // created before the nodes can enter maintenance. When the maintenance nodes
+  // return, the excess replicas should be removed.
+  public void testContainerIsReplicatedWhenAllNodesGotoMaintenance()
+      throws Exception {
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    // Locate any container and find its open pipeline
+    final ContainerInfo container = waitForAndReturnContainer();
+    Set<ContainerReplica> replicas = getContainerReplicas(container);
+
+    List<DatanodeDetails> forMaintenance = new ArrayList<>();
+    replicas.forEach(r ->forMaintenance.add(r.getDatanodeDetails()));
+
+    scmClient.startMaintenanceNodes(forMaintenance.stream()
+        .map(d -> getDNHostAndPort(d))
+        .collect(Collectors.toList()), 0);
+
+    // Ensure all 3 DNs go to maintenance
+    for(DatanodeDetails dn : forMaintenance) {
+      waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+    }
+
+    // There should now be 5 replicas of the container we are tracking
+    Set<ContainerReplica> newReplicas =
+        cm.getContainerReplicas(container.containerID());
+    assertEquals(5, newReplicas.size());
+
+    scmClient.recommissionNodes(forMaintenance.stream()
+        .map(d -> getDNHostAndPort(d))
+        .collect(Collectors.toList()));
+
+    // Ensure all 3 DNs go to maintenance
+    for(DatanodeDetails dn : forMaintenance) {
+      waitForDnToReachOpState(dn, IN_SERVICE);
+    }
+
+    GenericTestUtils.waitFor(() -> getContainerReplicas(container).size() == 3,
+        200, 30000);
+  }
+
+  @Test
+  // If SCM is restarted when a node is ENTERING_MAINTENANCE, then when the node
+  // re-registers, it should continue to enter maintenance.
+  public void testEnteringMaintenanceNodeCompletesAfterSCMRestart()
+      throws Exception {
+    // Stop Replication Manager to sure no containers are replicated
+    scm.getReplicationManager().stop();
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    // Locate any container and find its open pipeline
+    final ContainerInfo container = waitForAndReturnContainer();
+    Set<ContainerReplica> replicas = getContainerReplicas(container);
+
+    List<DatanodeDetails> forMaintenance = new ArrayList<>();
+    replicas.forEach(r ->forMaintenance.add(r.getDatanodeDetails()));
+
+    scmClient.startMaintenanceNodes(forMaintenance.stream()
+        .map(d -> getDNHostAndPort(d))
+        .collect(Collectors.toList()), 0);
+
+    // Ensure all 3 DNs go to maintenance
+    for(DatanodeDetails dn : forMaintenance) {
+      waitForDnToReachOpState(dn, IN_MAINTENANCE);
+    }
+
+    cluster.restartStorageContainerManager(true);
+
+    List<DatanodeDetails> newDns = new ArrayList<>();
+    for(DatanodeDetails dn : forMaintenance) {
+      newDns.add(nm.getNodeByUuid(dn.getUuid().toString()));
+    }
+
+    // Ensure all 3 DNs go to maintenance
+    for(DatanodeDetails dn : forMaintenance) {
+      waitForDnToReachOpState(dn, IN_MAINTENANCE);
+    }
+
+    // There should now be 5 replicas of the container we are tracking
+    Set<ContainerReplica> newReplicas =
+        cm.getContainerReplicas(container.containerID());
+    assertEquals(5, newReplicas.size());
+  }
+
+  @Test
+  // For a node which is online the maintenance should end automatically when
+  // maintenance expires and the node should go back into service.
+  // If the node is dead when maintenance expires, its replicas will be purge
+  // and new replicas created.
+  public void testMaintenanceEndsAutomaticallyAtTimeout()
+      throws Exception {
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    ContainerInfo container = waitForAndReturnContainer();
+    DatanodeDetails dn =
+        getOneDNHostingReplica(getContainerReplicas(container));
+
+    scmClient.startMaintenanceNodes(Arrays.asList(getDNHostAndPort(dn)), 0);
+    waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+
+    long newEndTime = System.currentTimeMillis() / 1000 + 5;
+    // Update the maintenance end time via NM manually. As the current
+    // decommission interface only allows us to specify hours from now as the
+    // end time, that is not really suitable for a test like this.
+    nm.setNodeOperationalState(dn, IN_MAINTENANCE, newEndTime);
+    waitForDnToReachOpState(dn, IN_SERVICE);
+    waitForDnToReachPersistedOpState(dn, IN_SERVICE);
+
+    // Put the node back into maintenance and then stop it and wait for it to
+    // go dead
+    scmClient.startMaintenanceNodes(Arrays.asList(getDNHostAndPort(dn)), 0);
+    waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+    cluster.shutdownHddsDatanode(dn);
+    waitForDnToReachHealthState(dn, DEAD);
+
+    newEndTime = System.currentTimeMillis() / 1000 + 5;
+    nm.setNodeOperationalState(dn, IN_MAINTENANCE, newEndTime);
+    waitForDnToReachOpState(dn, IN_SERVICE);
+    // Ensure there are 3 replicas not including the dead node, indicating a new
+    // replica was created
+    GenericTestUtils.waitFor(() -> getContainerReplicas(container)
+            .stream()
+            .filter(r -> !r.getDatanodeDetails().equals(dn))
+            .count() == 3,
+        200, 30000);
+  }
+
+  @Test
+  // If is SCM is Restarted when a maintenance node is dead, then we lose all
+  // the replicas associated with it, as the dead node cannot report them back
+  // in. If that happens, SCM has no choice except to replicate the containers.
+  public void testSCMHandlesRestartForMaintenanceNode()
+      throws Exception {
+    // Generate some data on the empty cluster to create some containers
+    generateData(20, "key", ReplicationFactor.THREE, ReplicationType.RATIS);
+    ContainerInfo container = waitForAndReturnContainer();
+    DatanodeDetails dn =
+        getOneDNHostingReplica(getContainerReplicas(container));
+
+    scmClient.startMaintenanceNodes(Arrays.asList(getDNHostAndPort(dn)), 0);
+    waitForDnToReachPersistedOpState(dn, IN_MAINTENANCE);
+
+    cluster.restartStorageContainerManager(true);
+    setManagers();
+
+    // Ensure there are 3 replicas with one in maintenance indicating no new
+    // replicas were created
+    final ContainerInfo newContainer = cm.getContainer(container.containerID());
+    GenericTestUtils.waitFor(() ->
+        getContainerReplicas(newContainer).size() == 3, 200, 30000);
+
+    ContainerReplicaCount counts =
+        scm.getReplicationManager().getContainerReplicaCount(newContainer);
+    assertEquals(1, counts.getMaintenanceCount());
+    assertTrue(counts.isSufficientlyReplicated());
+
+    // The node should be added back to the decommission monitor to ensure
+    // maintenance end time is correctly tracked.
+    GenericTestUtils.waitFor(() -> scm.getScmDecommissionManager().getMonitor()
+        .getTrackedNodes().size() == 1, 200, 30000);
+
+    // Now let the node go dead and repeat the test. This time ensure a new
+    // replica is created.
+    cluster.shutdownHddsDatanode(dn);
+    waitForDnToReachHealthState(dn, DEAD);
+
+    cluster.restartStorageContainerManager(false);
+    setManagers();
+
+    GenericTestUtils.waitFor(()
+        -> nm.getNodeCount(IN_SERVICE, null) == 5, 200, 30000);
+
+    // Ensure there are 3 replicas not including the dead node, indicating a new
+    // replica was created
+    final ContainerInfo nextContainer
+        = cm.getContainer(container.containerID());
+    GenericTestUtils.waitFor(() ->
+        getContainerReplicas(nextContainer).size() == 3, 200, 30000);
+    // There should be no IN_MAINTENANCE node:
+    assertEquals(0, nm.getNodeCount(IN_MAINTENANCE, null));
+    counts = scm.getReplicationManager().getContainerReplicaCount(newContainer);
+    assertEquals(0, counts.getMaintenanceCount());
+    assertTrue(counts.isSufficientlyReplicated());
+  }
+
+  /**
+   * Sets the instance variables to the values for the current MiniCluster.
+   */
+  private void setManagers() {
+    scm = cluster.getStorageContainerManager();
+    nm = scm.getScmNodeManager();
+    cm = scm.getContainerManager();
+    pm = scm.getPipelineManager();
+  }
+
+  /**
+   * Generates some data on the cluster so the cluster has some containers.
+   * @param keyCount The number of keys to create
+   * @param keyPrefix The prefix to use for the key name.
+   * @param repFactor The replication Factor for the keys
+   * @param repType The replication Type for the keys
+   * @throws IOException
+   */
+  private void generateData(int keyCount, String keyPrefix,
+      ReplicationFactor repFactor, ReplicationType repType) throws IOException {
+    for (int i=0; i<keyCount; i++) {
+      TestDataUtil.createKey(bucket, keyPrefix + i, repFactor, repType,
+          "this is the content");
+    }
+  }
+
+  /**
+   * Retrieves the NodeStatus for the given DN or fails the test if the
+   * Node cannot be found. This is a helper method to allow the nodeStatus to be
+   * checked in lambda expressions.
+   * @param dn Datanode for which to retrieve the NodeStatus.
+   * @return
+   */
+  private NodeStatus getNodeStatus(DatanodeDetails dn) {
+    NodeStatus status = null;
+    try {
+      status = nm.getNodeStatus(dn);
+    } catch (NodeNotFoundException e) {
+      fail("Unexpected exception getting the nodeState");
+    }
+    return status;
+  }
+
+  /**
+   * Retrieves the containerReplica set for a given container or fails the test
+   * if the container cannot be found. This is a helper method to allow the
+   * container replica count to be checked in a lambda expression.
+   * @param c The container for which to retrieve replicas
+   * @return
+   */
+  private Set<ContainerReplica> getContainerReplicas(ContainerInfo c) {
+    Set<ContainerReplica> replicas = null;
+    try {
+      replicas = cm.getContainerReplicas(c.containerID());
+    } catch (ContainerNotFoundException e) {
+      fail("Unexpected ContainerNotFoundException");
+    }
+    return replicas;
+  }
+
+  /**
+   * Select any DN hosting a replica from the Replica Set.
+   * @param replicas The set of ContainerReplica
+   * @return Any datanode associated one of the replicas
+   */
+  private DatanodeDetails getOneDNHostingReplica(
+      Set<ContainerReplica> replicas) {
+    // Now Decommission a host with one of the replicas
+    Iterator<ContainerReplica> iter = replicas.iterator();
+    ContainerReplica c = iter.next();
+    return c.getDatanodeDetails();
+  }
+
+  /**
+   * Given a Datanode, return a string consisting of the hostname and one of its
+   * ports in the for host:post.
+   * @param dn Datanode for which to retrieve the host:post string
+   * @return host:port for the given DN.
+   */
+  private String getDNHostAndPort(DatanodeDetails dn) {
+    return dn.getHostName()+":"+dn.getPorts().get(0).getValue();
+  }
+
+  /**
+   * Wait for the given datanode to reach the given operational state.
+   * @param dn Datanode for which to check the state
+   * @param state The state to wait for.
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private void waitForDnToReachOpState(DatanodeDetails dn,
+      HddsProtos.NodeOperationalState state)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(
+        () -> getNodeStatus(dn).getOperationalState().equals(state),
+        200, 30000);
+  }
+
+  /**
+   * Wait for the given datanode to reach the given Health state.
+   * @param dn Datanode for which to check the state
+   * @param state The state to wait for.
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private void waitForDnToReachHealthState(DatanodeDetails dn,
+      HddsProtos.NodeState state)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(
+        () -> getNodeStatus(dn).getHealth().equals(state),
+        200, 30000);
+  }
+
+  /**
+   * Wait for the given datanode to reach the given persisted state.
+   * @param dn Datanode for which to check the state
+   * @param state The state to wait for.
+   * @throws TimeoutException
+   * @throws InterruptedException
+   */
+  private void waitForDnToReachPersistedOpState(DatanodeDetails dn,
+      HddsProtos.NodeOperationalState state)
+      throws TimeoutException, InterruptedException {
+    GenericTestUtils.waitFor(
+        () -> dn.getPersistedOpState().equals(state),
+        200, 30000);
+  }
+
+  /**
+   * Get any container present in the cluster and wait to ensure 3 replicas
+   * have been reported before returning the container.
+   * @return A single container present on the cluster
+   * @throws Exception
+   */
+  private ContainerInfo waitForAndReturnContainer() throws Exception {
+    final ContainerInfo container = cm.getContainers().get(0);
+    // Ensure all 3 replicas of the container have been reported via ICR
+    GenericTestUtils.waitFor(
+        () -> getContainerReplicas(container).size() == 3,
+        200, 30000);
+    return container;
+  }
+
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java
index 5ac3a2b..2d25512 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/node/TestQueryNode.java
@@ -16,6 +16,9 @@
  */
 package org.apache.hadoop.ozone.scm.node;
 
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.node.NodeManager;
+import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.scm.ScmConfigKeys;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -46,6 +49,12 @@
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.DEAD;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.HEALTHY;
 import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState.STALE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.
+    NodeOperationalState.IN_SERVICE;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.
+    NodeOperationalState.DECOMMISSIONING;
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.
+    NodeOperationalState.IN_MAINTENANCE;
 
 import static org.apache.hadoop.hdds.scm.ScmConfigKeys
     .OZONE_SCM_DEADNODE_INTERVAL;
@@ -98,7 +107,7 @@
 
   @Test
   public void testHealthyNodesCount() throws Exception {
-    List<HddsProtos.Node> nodes = scmClient.queryNode(HEALTHY,
+    List<HddsProtos.Node> nodes = scmClient.queryNode(null, HEALTHY,
         HddsProtos.QueryScope.CLUSTER, "");
     assertEquals("Expected  live nodes", numOfDatanodes,
         nodes.size());
@@ -113,7 +122,7 @@
             cluster.getStorageContainerManager().getNodeCount(STALE) == 2,
         100, 4 * 1000);
 
-    int nodeCount = scmClient.queryNode(STALE,
+    int nodeCount = scmClient.queryNode(null, STALE,
         HddsProtos.QueryScope.CLUSTER, "").size();
     assertEquals("Mismatch of expected nodes count", 2, nodeCount);
 
@@ -122,13 +131,63 @@
         100, 4 * 1000);
 
     // Assert that we don't find any stale nodes.
-    nodeCount = scmClient.queryNode(STALE,
+    nodeCount = scmClient.queryNode(null, STALE,
         HddsProtos.QueryScope.CLUSTER, "").size();
     assertEquals("Mismatch of expected nodes count", 0, nodeCount);
 
     // Assert that we find the expected number of dead nodes.
-    nodeCount = scmClient.queryNode(DEAD,
+    nodeCount = scmClient.queryNode(null, DEAD,
         HddsProtos.QueryScope.CLUSTER, "").size();
     assertEquals("Mismatch of expected nodes count", 2, nodeCount);
   }
+
+  @Test
+  public void testNodeOperationalStates() throws Exception {
+    StorageContainerManager scm = cluster.getStorageContainerManager();
+    NodeManager nm = scm.getScmNodeManager();
+
+    // Set one node to be something other than IN_SERVICE
+    DatanodeDetails node = nm.getAllNodes().get(0);
+    nm.setNodeOperationalState(node, DECOMMISSIONING);
+
+    // All nodes should be returned as they are all in service
+    int nodeCount = scmClient.queryNode(IN_SERVICE, HEALTHY,
+        HddsProtos.QueryScope.CLUSTER, "").size();
+    assertEquals(numOfDatanodes - 1, nodeCount);
+
+    // null acts as wildcard for opState
+    nodeCount = scmClient.queryNode(null, HEALTHY,
+        HddsProtos.QueryScope.CLUSTER, "").size();
+    assertEquals(numOfDatanodes, nodeCount);
+
+    // null acts as wildcard for nodeState
+    nodeCount = scmClient.queryNode(IN_SERVICE, null,
+        HddsProtos.QueryScope.CLUSTER, "").size();
+    assertEquals(numOfDatanodes - 1, nodeCount);
+
+    // Both null - should return all nodes
+    nodeCount = scmClient.queryNode(null, null,
+        HddsProtos.QueryScope.CLUSTER, "").size();
+    assertEquals(numOfDatanodes, nodeCount);
+
+    // No node should be returned
+    nodeCount = scmClient.queryNode(IN_MAINTENANCE, HEALTHY,
+        HddsProtos.QueryScope.CLUSTER, "").size();
+    assertEquals(0, nodeCount);
+
+    // Test all operational states by looping over them all and setting the
+    // state manually.
+    node = nm.getAllNodes().get(0);
+    for (HddsProtos.NodeOperationalState s :
+        HddsProtos.NodeOperationalState.values()) {
+      nm.setNodeOperationalState(node, s);
+      nodeCount = scmClient.queryNode(s, HEALTHY,
+          HddsProtos.QueryScope.CLUSTER, "").size();
+      if (s == IN_SERVICE) {
+        assertEquals(5, nodeCount);
+      } else {
+        assertEquals(1, nodeCount);
+      }
+    }
+  }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 5a30910..9f57bec 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -47,7 +47,6 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Optional;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
@@ -175,16 +174,18 @@
 import org.apache.hadoop.util.KMSUtil;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ShutdownHookManager;
+import org.apache.hadoop.util.Time;
+
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.ObjectReader;
 import com.fasterxml.jackson.databind.ObjectWriter;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.ProtocolMessageEnum;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.tuple.Pair;
-
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_BLOCK_TOKEN_ENABLED_DEFAULT;
 import static org.apache.hadoop.hdds.HddsUtils.getScmAddressForBlockClients;
@@ -228,8 +229,6 @@
 import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
 import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneManagerService.newReflectiveBlockingService;
-
-import org.apache.hadoop.util.Time;
 import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
 import org.apache.ratis.server.protocol.TermIndex;
 import org.apache.ratis.util.ExitUtils;
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ClusterStateEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ClusterStateEndpoint.java
index de0028c..3b94b0b 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ClusterStateEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ClusterStateEndpoint.java
@@ -19,8 +19,8 @@
 package org.apache.hadoop.ozone.recon.api;
 
 import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
 import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
+import org.apache.hadoop.hdds.scm.node.NodeStatus;
 import org.apache.hadoop.hdds.scm.server.OzoneStorageContainerManager;
 import org.apache.hadoop.ozone.recon.api.types.ClusterStateResponse;
 import org.apache.hadoop.ozone.recon.api.types.DatanodeStorageReport;
@@ -80,7 +80,8 @@
     List<DatanodeDetails> datanodeDetails = nodeManager.getAllNodes();
     int containers = this.containerManager.getContainerIDs().size();
     int pipelines = this.pipelineManager.getPipelines().size();
-    int healthyDatanodes = nodeManager.getNodeCount(NodeState.HEALTHY);
+    int healthyDatanodes =
+        nodeManager.getNodeCount(NodeStatus.inServiceHealthy());
     SCMNodeStat stats = nodeManager.getStats();
     DatanodeStorageReport storageReport =
         new DatanodeStorageReport(stats.getCapacity().get(),
diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
index bd022c4..2f88497 100644
--- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
+++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/NodeEndpoint.java
@@ -79,7 +79,12 @@
 
     datanodeDetails.forEach(datanode -> {
       DatanodeStorageReport storageReport = getStorageReport(datanode);
-      NodeState nodeState = nodeManager.getNodeState(datanode);
+      NodeState nodeState = null;
+      try {
+        nodeState = nodeManager.getNodeStatus(datanode).getHealth();
+      } catch (NodeNotFoundException e) {
+        LOG.warn("Cannot get nodeState for datanode {}", datanode, e);
+      }
       String hostname = datanode.getHostName();
       Set<PipelineID> pipelineIDs = nodeManager.getPipelines(datanode);
       List<DatanodePipeline> pipelines = new ArrayList<>();