You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by me...@apache.org on 2019/11/18 14:20:53 UTC

[hbase] branch branch-2.2 updated: HBASE-22982: region server suspend/resume and graceful rolling restart actions (#824)

This is an automated email from the ASF dual-hosted git repository.

meszibalu pushed a commit to branch branch-2.2
in repository https://gitbox.apache.org/repos/asf/hbase.git


The following commit(s) were added to refs/heads/branch-2.2 by this push:
     new 1eceb24  HBASE-22982: region server suspend/resume and graceful rolling restart actions (#824)
1eceb24 is described below

commit 1eceb24b671d02899087535e806742428affb13a
Author: BukrosSzabolcs <bu...@gmail.com>
AuthorDate: Mon Nov 18 06:20:39 2019 -0800

    HBASE-22982: region server suspend/resume and graceful rolling restart actions (#824)
    
    * Add chaos monkey action for suspend/resume region servers
    * Add chaos monkey action for graceful rolling restart
    * Add these to relevant chaos monkeys
    
    Signed-off-by: Balazs Meszaros <me...@apache.org>
    Signed-off-by: Peter Somogyi <ps...@apache.org>
---
 .../hadoop/hbase/DistributedHBaseCluster.java      |  82 +++++++++------
 .../apache/hadoop/hbase/chaos/actions/Action.java  |  90 +++++++++++-----
 .../actions/GracefulRollingRestartRsAction.java    |  73 +++++++++++++
 .../chaos/actions/RestartActionBaseAction.java     |  29 +++++
 .../actions/RollingBatchSuspendResumeRsAction.java | 117 +++++++++++++++++++++
 .../hbase/chaos/factories/MonkeyConstants.java     |   6 ++
 .../ServerAndDependenciesKillingMonkeyFactory.java |  24 ++++-
 .../factories/ServerKillingMonkeyFactory.java      |  24 ++++-
 .../factories/SlowDeterministicMonkeyFactory.java  |  17 +++
 .../StressAssignmentManagerMonkeyFactory.java      |  23 ++++
 .../chaos/monkies/PolicyBasedChaosMonkey.java      |  18 ++--
 .../java/org/apache/hadoop/hbase/HBaseCluster.java |  14 +++
 .../org/apache/hadoop/hbase/MiniHBaseCluster.java  |  36 +++++++
 13 files changed, 480 insertions(+), 73 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 30a3db9..57e929b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -113,13 +113,13 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public void startRegionServer(String hostname, int port) throws IOException {
-    LOG.info("Starting RS on: " + hostname);
+    LOG.info("Starting RS on: {}", hostname);
     clusterManager.start(ServiceType.HBASE_REGIONSERVER, hostname, port);
   }
 
   @Override
   public void killRegionServer(ServerName serverName) throws IOException {
-    LOG.info("Aborting RS: " + serverName.getServerName());
+    LOG.info("Aborting RS: {}", serverName.getServerName());
     killedRegionServers.add(serverName);
     clusterManager.kill(ServiceType.HBASE_REGIONSERVER,
       serverName.getHostname(), serverName.getPort());
@@ -132,7 +132,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public void stopRegionServer(ServerName serverName) throws IOException {
-    LOG.info("Stopping RS: " + serverName.getServerName());
+    LOG.info("Stopping RS: {}", serverName.getServerName());
     clusterManager.stop(ServiceType.HBASE_REGIONSERVER,
       serverName.getHostname(), serverName.getPort());
   }
@@ -143,21 +143,35 @@ public class DistributedHBaseCluster extends HBaseCluster {
   }
 
   @Override
+  public void suspendRegionServer(ServerName serverName) throws IOException {
+    LOG.info("Suspend RS: {}", serverName.getServerName());
+    clusterManager.suspend(ServiceType.HBASE_REGIONSERVER,
+        serverName.getHostname(), serverName.getPort());
+  }
+
+  @Override
+  public void resumeRegionServer(ServerName serverName) throws IOException {
+    LOG.info("Resume RS: {}", serverName.getServerName());
+    clusterManager.resume(ServiceType.HBASE_REGIONSERVER,
+        serverName.getHostname(), serverName.getPort());
+  }
+
+  @Override
   public void startZkNode(String hostname, int port) throws IOException {
-    LOG.info("Starting ZooKeeper node on: " + hostname);
+    LOG.info("Starting ZooKeeper node on: {}", hostname);
     clusterManager.start(ServiceType.ZOOKEEPER_SERVER, hostname, port);
   }
 
   @Override
   public void killZkNode(ServerName serverName) throws IOException {
-    LOG.info("Aborting ZooKeeper node on: " + serverName.getServerName());
+    LOG.info("Aborting ZooKeeper node on: {}", serverName.getServerName());
     clusterManager.kill(ServiceType.ZOOKEEPER_SERVER,
       serverName.getHostname(), serverName.getPort());
   }
 
   @Override
   public void stopZkNode(ServerName serverName) throws IOException {
-    LOG.info("Stopping ZooKeeper node: " + serverName.getServerName());
+    LOG.info("Stopping ZooKeeper node: {}", serverName.getServerName());
     clusterManager.stop(ServiceType.ZOOKEEPER_SERVER,
       serverName.getHostname(), serverName.getPort());
   }
@@ -174,21 +188,21 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public void startDataNode(ServerName serverName) throws IOException {
-    LOG.info("Starting data node on: " + serverName.getServerName());
+    LOG.info("Starting data node on: {}", serverName.getServerName());
     clusterManager.start(ServiceType.HADOOP_DATANODE,
       serverName.getHostname(), serverName.getPort());
   }
 
   @Override
   public void killDataNode(ServerName serverName) throws IOException {
-    LOG.info("Aborting data node on: " + serverName.getServerName());
+    LOG.info("Aborting data node on: {}", serverName.getServerName());
     clusterManager.kill(ServiceType.HADOOP_DATANODE,
       serverName.getHostname(), serverName.getPort());
   }
 
   @Override
   public void stopDataNode(ServerName serverName) throws IOException {
-    LOG.info("Stopping data node on: " + serverName.getServerName());
+    LOG.info("Stopping data node on: {}", serverName.getServerName());
     clusterManager.stop(ServiceType.HADOOP_DATANODE,
       serverName.getHostname(), serverName.getPort());
   }
@@ -205,21 +219,21 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public void startNameNode(ServerName serverName) throws IOException {
-    LOG.info("Starting name node on: " + serverName.getServerName());
+    LOG.info("Starting name node on: {}", serverName.getServerName());
     clusterManager.start(ServiceType.HADOOP_NAMENODE, serverName.getHostname(),
       serverName.getPort());
   }
 
   @Override
   public void killNameNode(ServerName serverName) throws IOException {
-    LOG.info("Aborting name node on: " + serverName.getServerName());
+    LOG.info("Aborting name node on: {}", serverName.getServerName());
     clusterManager.kill(ServiceType.HADOOP_NAMENODE, serverName.getHostname(),
       serverName.getPort());
   }
 
   @Override
   public void stopNameNode(ServerName serverName) throws IOException {
-    LOG.info("Stopping name node on: " + serverName.getServerName());
+    LOG.info("Stopping name node on: {}", serverName.getServerName());
     clusterManager.stop(ServiceType.HADOOP_NAMENODE, serverName.getHostname(),
       serverName.getPort());
   }
@@ -236,7 +250,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   private void waitForServiceToStop(ServiceType service, ServerName serverName, long timeout)
     throws IOException {
-    LOG.info("Waiting for service: " + service + " to stop: " + serverName.getServerName());
+    LOG.info("Waiting for service: {} to stop: {}", service, serverName.getServerName());
     long start = System.currentTimeMillis();
 
     while ((System.currentTimeMillis() - start) < timeout) {
@@ -250,7 +264,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   private void waitForServiceToStart(ServiceType service, ServerName serverName, long timeout)
     throws IOException {
-    LOG.info("Waiting for service: " + service + " to start: " + serverName.getServerName());
+    LOG.info("Waiting for service: {} to start: ", service, serverName.getServerName());
     long start = System.currentTimeMillis();
 
     while ((System.currentTimeMillis() - start) < timeout) {
@@ -271,19 +285,19 @@ public class DistributedHBaseCluster extends HBaseCluster {
 
   @Override
   public void startMaster(String hostname, int port) throws IOException {
-    LOG.info("Starting Master on: " + hostname + ":" + port);
+    LOG.info("Starting Master on: {}:{}", hostname, port);
     clusterManager.start(ServiceType.HBASE_MASTER, hostname, port);
   }
 
   @Override
   public void killMaster(ServerName serverName) throws IOException {
-    LOG.info("Aborting Master: " + serverName.getServerName());
+    LOG.info("Aborting Master: {}", serverName.getServerName());
     clusterManager.kill(ServiceType.HBASE_MASTER, serverName.getHostname(), serverName.getPort());
   }
 
   @Override
   public void stopMaster(ServerName serverName) throws IOException {
-    LOG.info("Stopping Master: " + serverName.getServerName());
+    LOG.info("Stopping Master: {}", serverName.getServerName());
     clusterManager.stop(ServiceType.HBASE_MASTER, serverName.getHostname(), serverName.getPort());
   }
 
@@ -317,7 +331,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       regionLoc = locator.getRegionLocation(startKey, true);
     }
     if (regionLoc == null) {
-      LOG.warn("Cannot find region server holding region " + Bytes.toStringBinary(regionName));
+      LOG.warn("Cannot find region server holding region {}", Bytes.toStringBinary(regionName));
       return null;
     }
     return regionLoc.getServerName();
@@ -361,15 +375,15 @@ public class DistributedHBaseCluster extends HBaseCluster {
     //check whether current master has changed
     final ServerName initMaster = initial.getMasterName();
     if (!ServerName.isSameAddress(initMaster, current.getMasterName())) {
-      LOG.info("Restoring cluster - Initial active master : " + initMaster.getAddress() +
-        " has changed to : " + current.getMasterName().getAddress());
+      LOG.info("Restoring cluster - Initial active master : {} has changed to : {}",
+          initMaster.getAddress(), current.getMasterName().getAddress());
       // If initial master is stopped, start it, before restoring the state.
       // It will come up as a backup master, if there is already an active master.
       try {
         if (!clusterManager.isRunning(ServiceType.HBASE_MASTER,
                 initMaster.getHostname(), initMaster.getPort())) {
-          LOG.info("Restoring cluster - starting initial active master at:"
-                  + initMaster.getAddress());
+          LOG.info("Restoring cluster - starting initial active master at:{}",
+              initMaster.getAddress());
           startMaster(initMaster.getHostname(), initMaster.getPort());
         }
 
@@ -379,11 +393,11 @@ public class DistributedHBaseCluster extends HBaseCluster {
         // 3. Start backup masters
         for (ServerName currentBackup : current.getBackupMasterNames()) {
           if (!ServerName.isSameAddress(currentBackup, initMaster)) {
-            LOG.info("Restoring cluster - stopping backup master: " + currentBackup);
+            LOG.info("Restoring cluster - stopping backup master: {}", currentBackup);
             stopMaster(currentBackup);
           }
         }
-        LOG.info("Restoring cluster - stopping active master: " + current.getMasterName());
+        LOG.info("Restoring cluster - stopping active master: {}", current.getMasterName());
         stopMaster(current.getMasterName());
         waitForActiveAndReadyMaster(); // wait so that active master takes over
       } catch (IOException ex) {
@@ -399,8 +413,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
           if (!clusterManager.isRunning(ServiceType.HBASE_MASTER,
                   backup.getHostname(),
                   backup.getPort())) {
-            LOG.info("Restoring cluster - starting initial backup master: "
-                    + backup.getAddress());
+            LOG.info("Restoring cluster - starting initial backup master: {}",
+                backup.getAddress());
             startMaster(backup.getHostname(), backup.getPort());
           }
         } catch (IOException ex) {
@@ -424,7 +438,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       for (ServerName sn:toStart) {
         try {
           if(!clusterManager.isRunning(ServiceType.HBASE_MASTER, sn.getHostname(), sn.getPort())) {
-            LOG.info("Restoring cluster - starting initial backup master: " + sn.getAddress());
+            LOG.info("Restoring cluster - starting initial backup master: {}", sn.getAddress());
             startMaster(sn.getHostname(), sn.getPort());
           }
         } catch (IOException ex) {
@@ -435,7 +449,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       for (ServerName sn:toKill) {
         try {
           if(clusterManager.isRunning(ServiceType.HBASE_MASTER, sn.getHostname(), sn.getPort())) {
-            LOG.info("Restoring cluster - stopping backup master: " + sn.getAddress());
+            LOG.info("Restoring cluster - stopping backup master: {}", sn.getAddress());
             stopMaster(sn);
           }
         } catch (IOException ex) {
@@ -444,8 +458,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
       }
     }
     if (!deferred.isEmpty()) {
-      LOG.warn("Restoring cluster - restoring region servers reported "
-              + deferred.size() + " errors:");
+      LOG.warn("Restoring cluster - restoring region servers reported {} errors:",
+          deferred.size());
       for (int i=0; i<deferred.size() && i < 3; i++) {
         LOG.warn(Objects.toString(deferred.get(i)));
       }
@@ -487,7 +501,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       try {
         if (!clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER, sn.getHostname(),
           sn.getPort()) && master.getPort() != sn.getPort()) {
-          LOG.info("Restoring cluster - starting initial region server: " + sn.getAddress());
+          LOG.info("Restoring cluster - starting initial region server: {}", sn.getAddress());
           startRegionServer(sn.getHostname(), sn.getPort());
         }
       } catch (IOException ex) {
@@ -499,7 +513,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
       try {
         if (clusterManager.isRunning(ServiceType.HBASE_REGIONSERVER, sn.getHostname(),
           sn.getPort()) && master.getPort() != sn.getPort()) {
-          LOG.info("Restoring cluster - stopping initial region server: " + sn.getAddress());
+          LOG.info("Restoring cluster - stopping initial region server: {}", sn.getAddress());
           stopRegionServer(sn);
         }
       } catch (IOException ex) {
@@ -507,8 +521,8 @@ public class DistributedHBaseCluster extends HBaseCluster {
       }
     }
     if (!deferred.isEmpty()) {
-      LOG.warn("Restoring cluster - restoring region servers reported "
-              + deferred.size() + " errors:");
+      LOG.warn("Restoring cluster - restoring region servers reported {} errors:",
+          deferred.size());
       for (int i=0; i<deferred.size() && i < 3; i++) {
         LOG.warn(Objects.toString(deferred.get(i)));
       }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
index 58abea6..be4251a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/Action.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
 import org.apache.hadoop.hbase.ServerMetrics;
 import org.apache.hadoop.hbase.ServerName;
 import org.apache.hadoop.hbase.TableName;
@@ -150,78 +151,106 @@ public class Action {
   }
 
   protected void killMaster(ServerName server) throws IOException {
-    LOG.info("Killing master " + server);
+    LOG.info("Killing master {}", server);
     cluster.killMaster(server);
     cluster.waitForMasterToStop(server, killMasterTimeout);
     LOG.info("Killed master " + server);
   }
 
   protected void startMaster(ServerName server) throws IOException {
-    LOG.info("Starting master " + server.getHostname());
+    LOG.info("Starting master {}", server.getHostname());
     cluster.startMaster(server.getHostname(), server.getPort());
     cluster.waitForActiveAndReadyMaster(startMasterTimeout);
     LOG.info("Started master " + server.getHostname());
   }
 
+  protected void stopRs(ServerName server) throws IOException {
+    LOG.info("Stopping regionserver {}", server);
+    cluster.stopRegionServer(server);
+    cluster.waitForRegionServerToStop(server, killRsTimeout);
+    LOG.info("Stoppiong regionserver {}. Reported num of rs:{}", server,
+        cluster.getClusterMetrics().getLiveServerMetrics().size());
+  }
+
+  protected void suspendRs(ServerName server) throws IOException {
+    LOG.info("Suspending regionserver {}", server);
+    cluster.suspendRegionServer(server);
+    if(!(cluster instanceof MiniHBaseCluster)){
+      cluster.waitForRegionServerToStop(server, killRsTimeout);
+    }
+    LOG.info("Suspending regionserver {}. Reported num of rs:{}", server,
+        cluster.getClusterMetrics().getLiveServerMetrics().size());
+  }
+
+  protected void resumeRs(ServerName server) throws IOException {
+    LOG.info("Resuming regionserver {}", server);
+    cluster.resumeRegionServer(server);
+    if(!(cluster instanceof MiniHBaseCluster)){
+      cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
+    }
+    LOG.info("Resuming regionserver {}. Reported num of rs:{}", server,
+        cluster.getClusterMetrics().getLiveServerMetrics().size());
+  }
+
   protected void killRs(ServerName server) throws IOException {
-    LOG.info("Killing regionserver " + server);
+    LOG.info("Killing regionserver {}", server);
     cluster.killRegionServer(server);
     cluster.waitForRegionServerToStop(server, killRsTimeout);
-    LOG.info("Killed regionserver " + server + ". Reported num of rs:"
-        + cluster.getClusterMetrics().getLiveServerMetrics().size());
+    LOG.info("Killed regionserver {}. Reported num of rs:{}", server,
+        cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startRs(ServerName server) throws IOException {
-    LOG.info("Starting regionserver " + server.getAddress());
+    LOG.info("Starting regionserver {}", server.getAddress());
     cluster.startRegionServer(server.getHostname(), server.getPort());
     cluster.waitForRegionServerToStart(server.getHostname(), server.getPort(), startRsTimeout);
-    LOG.info("Started regionserver " + server.getAddress() + ". Reported num of rs:"
-      + cluster.getClusterMetrics().getLiveServerMetrics().size());
+    LOG.info("Started regionserver {}. Reported num of rs:{}", server.getAddress(),
+      cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void killZKNode(ServerName server) throws IOException {
-    LOG.info("Killing zookeeper node " + server);
+    LOG.info("Killing zookeeper node {}", server);
     cluster.killZkNode(server);
     cluster.waitForZkNodeToStop(server, killZkNodeTimeout);
-    LOG.info("Killed zookeeper node " + server + ". Reported num of rs:"
-      + cluster.getClusterMetrics().getLiveServerMetrics().size());
+    LOG.info("Killed zookeeper node {}. Reported num of rs:{}", server,
+      cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startZKNode(ServerName server) throws IOException {
-    LOG.info("Starting zookeeper node " + server.getHostname());
+    LOG.info("Starting zookeeper node {}", server.getHostname());
     cluster.startZkNode(server.getHostname(), server.getPort());
     cluster.waitForZkNodeToStart(server, startZkNodeTimeout);
-    LOG.info("Started zookeeper node " + server);
+    LOG.info("Started zookeeper node {}", server);
   }
 
   protected void killDataNode(ServerName server) throws IOException {
-    LOG.info("Killing datanode " + server);
+    LOG.info("Killing datanode {}", server);
     cluster.killDataNode(server);
     cluster.waitForDataNodeToStop(server, killDataNodeTimeout);
-    LOG.info("Killed datanode " + server + ". Reported num of rs:"
-      + cluster.getClusterMetrics().getLiveServerMetrics().size());
+    LOG.info("Killed datanode {}. Reported num of rs:{}", server,
+      cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startDataNode(ServerName server) throws IOException {
-    LOG.info("Starting datanode " + server.getHostname());
+    LOG.info("Starting datanode {}", server.getHostname());
     cluster.startDataNode(server);
     cluster.waitForDataNodeToStart(server, startDataNodeTimeout);
-    LOG.info("Started datanode " + server);
+    LOG.info("Started datanode {}", server);
   }
 
   protected void killNameNode(ServerName server) throws IOException {
-    LOG.info("Killing namenode :-" + server.getHostname());
+    LOG.info("Killing namenode :-{}", server.getHostname());
     cluster.killNameNode(server);
     cluster.waitForNameNodeToStop(server, killNameNodeTimeout);
-    LOG.info("Killed namenode:" + server + ". Reported num of rs:"
-        + cluster.getClusterMetrics().getLiveServerMetrics().size());
+    LOG.info("Killed namenode:{}. Reported num of rs:{}", server,
+        cluster.getClusterMetrics().getLiveServerMetrics().size());
   }
 
   protected void startNameNode(ServerName server) throws IOException {
-    LOG.info("Starting Namenode :-" + server.getHostname());
+    LOG.info("Starting Namenode :-{}", server.getHostname());
     cluster.startNameNode(server);
     cluster.waitForNameNodeToStart(server, startNameNodeTimeout);
-    LOG.info("Started namenode:" + server);
+    LOG.info("Started namenode:{}", server);
   }
   protected void unbalanceRegions(ClusterMetrics clusterStatus,
       List<ServerName> fromServers, List<ServerName> toServers,
@@ -234,7 +263,7 @@ public class Action {
       // Ugh.
       List<byte[]> regions = new LinkedList<>(serverLoad.getRegionMetrics().keySet());
       int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
-      LOG.debug("Removing " + victimRegionCount + " regions from " + sn);
+      LOG.debug("Removing {} regions from {}", victimRegionCount, sn);
       for (int i = 0; i < victimRegionCount; ++i) {
         int victimIx = RandomUtils.nextInt(0, regions.size());
         String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
@@ -242,8 +271,8 @@ public class Action {
       }
     }
 
-    LOG.info("Moving " + victimRegions.size() + " regions from " + fromServers.size()
-        + " servers to " + toServers.size() + " different servers");
+    LOG.info("Moving {} regions from {} servers to {} different servers", victimRegions.size(),
+        fromServers.size(), toServers.size());
     Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
     for (byte[] victimRegion : victimRegions) {
       // Don't keep moving regions if we're
@@ -269,6 +298,15 @@ public class Action {
     }
   }
 
+  protected void setBalancer(boolean onOrOff, boolean synchronous) throws Exception {
+    Admin admin = this.context.getHBaseIntegrationTestingUtility().getAdmin();
+    try {
+      admin.balancerSwitch(onOrOff, synchronous);
+    } catch (Exception e) {
+      LOG.warn("Got exception while switching balance ", e);
+    }
+  }
+
   public Configuration getConf() {
     return cluster.getConf();
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java
new file mode 100644
index 0000000..82005bb
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/GracefulRollingRestartRsAction.java
@@ -0,0 +1,73 @@
+/**
+ * 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.hbase.chaos.actions;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.util.RegionMover;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Gracefully restarts every regionserver in a rolling fashion. At each step, it unloads,
+ * restarts the loads every rs server sleeping randomly (0-sleepTime) in between servers.
+ */
+public class GracefulRollingRestartRsAction extends RestartActionBaseAction {
+  private static final Logger LOG = LoggerFactory.getLogger(GracefulRollingRestartRsAction.class);
+
+  public GracefulRollingRestartRsAction(long sleepTime) {
+    super(sleepTime);
+  }
+
+  @Override
+  public void perform() throws Exception {
+    LOG.info("Performing action: Rolling restarting non-master region servers");
+    List<ServerName> selectedServers = selectServers();
+
+    LOG.info("Disabling balancer to make unloading possible");
+    setBalancer(false, true);
+
+    for (ServerName server : selectedServers) {
+      String rsName = server.getAddress().toString();
+      try (RegionMover rm =
+          new RegionMover.RegionMoverBuilder(rsName, getConf()).ack(true).build()) {
+        LOG.info("Unloading {}", server);
+        rm.unload();
+        LOG.info("Restarting {}", server);
+        gracefulRestartRs(server, sleepTime);
+        LOG.info("Loading {}", server);
+        rm.load();
+      } catch (Shell.ExitCodeException e) {
+        LOG.info("Problem restarting but presume successful; code={}", e.getExitCode(), e);
+      }
+      sleep(RandomUtils.nextInt(0, (int)sleepTime));
+    }
+    LOG.info("Enabling balancer");
+    setBalancer(true, true);
+  }
+
+  protected List<ServerName> selectServers() throws IOException {
+    return Arrays.asList(getCurrentServers());
+  }
+
+}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java
index c36526f..c53de90 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RestartActionBaseAction.java
@@ -49,19 +49,42 @@ public class RestartActionBaseAction extends Action {
       return;
     }
 
+    LOG.info("Killing master: {}", server);
     killMaster(server);
     sleep(sleepTime);
+    LOG.info("Starting master: {}", server);
     startMaster(server);
   }
 
+  /**
+   * Stop and then restart the region server instead of killing it.
+   * @param server hostname to restart the regionserver on
+   * @param sleepTime number of milliseconds between stop and restart
+   * @throws IOException if something goes wrong
+   */
+  void gracefulRestartRs(ServerName server, long sleepTime) throws IOException {
+    sleepTime = Math.max(sleepTime, 1000);
+    // Don't try the stop if we're stopping already
+    if (context.isStopping()) {
+      return;
+    }
+    LOG.info("Stopping region server: {}", server);
+    stopRs(server);
+    sleep(sleepTime);
+    LOG.info("Starting region server: {}", server);
+    startRs(server);
+  }
+
   void restartRs(ServerName server, long sleepTime) throws IOException {
     sleepTime = Math.max(sleepTime, 1000);
     // Don't try the kill if we're stopping
     if (context.isStopping()) {
       return;
     }
+    LOG.info("Killing region server: {}", server);
     killRs(server);
     sleep(sleepTime);
+    LOG.info("Starting region server: {}", server);
     startRs(server);
   }
 
@@ -71,8 +94,10 @@ public class RestartActionBaseAction extends Action {
     if (context.isStopping()) {
       return;
     }
+    LOG.info("Killing zookeeper node: {}", server);
     killZKNode(server);
     sleep(sleepTime);
+    LOG.info("Starting zookeeper node: {}", server);
     startZKNode(server);
   }
 
@@ -82,8 +107,10 @@ public class RestartActionBaseAction extends Action {
     if (context.isStopping()) {
       return;
     }
+    LOG.info("Killing data node: {}", server);
     killDataNode(server);
     sleep(sleepTime);
+    LOG.info("Starting data node: {}", server);
     startDataNode(server);
   }
 
@@ -93,8 +120,10 @@ public class RestartActionBaseAction extends Action {
     if (context.isStopping()) {
       return;
     }
+    LOG.info("Killing name node: {}", server);
     killNameNode(server);
     sleep(sleepTime);
+    LOG.info("Starting name node: {}", server);
     startNameNode(server);
   }
 
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java
new file mode 100644
index 0000000..d4ad3e4
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchSuspendResumeRsAction.java
@@ -0,0 +1,117 @@
+/**
+ * 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.hbase.chaos.actions;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Queue;
+
+import org.apache.commons.lang3.RandomUtils;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
+import org.apache.hadoop.hbase.util.Threads;
+import org.apache.hadoop.util.Shell;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Suspend then resume a ratio of the regionservers in a rolling fashion. At each step, either
+ * suspend a server, or resume one, sleeping (sleepTime) in between steps. The parameter
+ * maxSuspendedServers limits the maximum number of servers that can be down at the same time
+ * during rolling restarts.
+ */
+public class RollingBatchSuspendResumeRsAction extends Action {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RollingBatchSuspendResumeRsAction.class);
+  private float ratio;
+  private long sleepTime;
+  private int maxSuspendedServers; // number of maximum suspended servers at any given time.
+
+  public RollingBatchSuspendResumeRsAction(long sleepTime, float ratio) {
+    this(sleepTime, ratio, 5);
+  }
+
+  public RollingBatchSuspendResumeRsAction(long sleepTime, float ratio, int maxSuspendedServers) {
+    this.ratio = ratio;
+    this.sleepTime = sleepTime;
+    this.maxSuspendedServers = maxSuspendedServers;
+  }
+
+  enum SuspendOrResume {
+    SUSPEND, RESUME
+  }
+
+  @Override
+  public void perform() throws Exception {
+    LOG.info(String.format("Performing action: Rolling batch restarting %d%% of region servers",
+        (int) (ratio * 100)));
+    List<ServerName> selectedServers = selectServers();
+
+    Queue<ServerName> serversToBeSuspended = new LinkedList<>(selectedServers);
+    Queue<ServerName> suspendedServers = new LinkedList<>();
+
+    // loop while there are servers to be suspended or suspended servers to be resumed
+    while ((!serversToBeSuspended.isEmpty() || !suspendedServers.isEmpty()) && !context
+        .isStopping()) {
+      SuspendOrResume action;
+
+      if (serversToBeSuspended.isEmpty()) { // no more servers to suspend
+        action = SuspendOrResume.RESUME;
+      } else if (suspendedServers.isEmpty()) {
+        action = SuspendOrResume.SUSPEND; // no more servers to resume
+      } else if (suspendedServers.size() >= maxSuspendedServers) {
+        // we have too many suspended servers. Don't suspend any more
+        action = SuspendOrResume.RESUME;
+      } else {
+        // do a coin toss
+        action = RandomUtils.nextBoolean() ? SuspendOrResume.SUSPEND : SuspendOrResume.RESUME;
+      }
+
+      ServerName server;
+      switch (action) {
+        case SUSPEND:
+          server = serversToBeSuspended.remove();
+          try {
+            suspendRs(server);
+          } catch (Shell.ExitCodeException e) {
+            LOG.warn("Problem suspending but presume successful; code={}", e.getExitCode(), e);
+          }
+          suspendedServers.add(server);
+          break;
+        case RESUME:
+          server = suspendedServers.remove();
+          try {
+            resumeRs(server);
+          } catch (Shell.ExitCodeException e) {
+            LOG.info("Problem resuming, will retry; code={}", e.getExitCode(), e);
+          }
+          break;
+      }
+
+      LOG.info("Sleeping for:{}", sleepTime);
+      Threads.sleep(sleepTime);
+    }
+  }
+
+  protected List<ServerName> selectServers() throws IOException {
+    return PolicyBasedChaosMonkey.selectRandomItems(getCurrentServers(), ratio);
+  }
+
+}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java
index 5657d39..9051e98 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java
@@ -45,6 +45,9 @@ public interface MonkeyConstants {
   String UNBALANCE_WAIT_AFTER_BALANCE_MS = "unbalance.action.wait.after.period";
   String UNBALANCE_KILL_META_RS = "unbalance.action.kill.meta.rs";
   String DECREASE_HFILE_SIZE_SLEEP_TIME = "decrease.hfile.size.sleep.time";
+  String GRACEFUL_RESTART_RS_SLEEP_TIME = "graceful.restart.rs.sleep.time";
+  String ROLLING_BATCH_SUSPEND_RS_SLEEP_TIME = "rolling.batch.suspend.rs.sleep.time";
+  String ROLLING_BATCH_SUSPEND_RS_RATIO = "rolling.batch.suspend.rs.ratio";
 
   /**
    * A Set of prefixes which encompasses all of the configuration properties for the ChaosMonky.
@@ -75,4 +78,7 @@ public interface MonkeyConstants {
   long DEFAULT_UNBALANCE_WAIT_AFTER_BALANCE_MS = 5 * 1000;
   boolean DEFAULT_UNBALANCE_KILL_META_RS = true;
   long DEFAULT_DECREASE_HFILE_SIZE_SLEEP_TIME = 30 * 1000;
+  long DEFAULT_GRACEFUL_RESTART_RS_SLEEP_TIME = 5000;
+  long DEFAULT_ROLLING_BATCH_SUSPEND_RS_SLEEP_TIME = 30 * 1000;
+  float DEFAULT_ROLLING_BATCH_SUSPEND_RS_RATIO = 1.0f;
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerAndDependenciesKillingMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerAndDependenciesKillingMonkeyFactory.java
index 4faa786..2e763ad 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerAndDependenciesKillingMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerAndDependenciesKillingMonkeyFactory.java
@@ -21,11 +21,13 @@ package org.apache.hadoop.hbase.chaos.factories;
 import org.apache.hadoop.hbase.chaos.actions.Action;
 import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.ForceBalancerAction;
+import org.apache.hadoop.hbase.chaos.actions.GracefulRollingRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomDataNodeAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomRsExceptMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomZKNodeAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchRestartRsExceptMetaAction;
+import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.chaos.policies.CompositeSequentialPolicy;
@@ -38,8 +40,13 @@ import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
  */
 public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
 
+  private long gracefulRollingRestartTSSLeepTime;
+  private long rollingBatchSuspendRSSleepTime;
+  private float rollingBatchSuspendtRSRatio;
+
   @Override
   public ChaosMonkey build() {
+    loadProperties();
 
     // Destructive actions to mess things around. Cannot run batch restart.
     Action[] actions1 = new Action[]{
@@ -48,7 +55,10 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
       new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), // only allow 2 servers to be dead.
       new ForceBalancerAction(),
       new RestartRandomDataNodeAction(60000),
-      new RestartRandomZKNodeAction(60000)
+      new RestartRandomZKNodeAction(60000),
+      new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
+      new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
+          rollingBatchSuspendtRSRatio)
     };
 
     // Action to log more info for debugging
@@ -62,4 +72,16 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
         new PeriodicRandomActionPolicy(60 * 1000, actions1)),
       new PeriodicRandomActionPolicy(60 * 1000, actions2));
   }
+
+  private void loadProperties() {
+    gracefulRollingRestartTSSLeepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.GRACEFUL_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_GRACEFUL_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendtRSRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
+  }
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerKillingMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerKillingMonkeyFactory.java
index 02b5914..68d11f9 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerKillingMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/ServerKillingMonkeyFactory.java
@@ -21,9 +21,11 @@ package org.apache.hadoop.hbase.chaos.factories;
 import org.apache.hadoop.hbase.chaos.actions.Action;
 import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.ForceBalancerAction;
+import org.apache.hadoop.hbase.chaos.actions.GracefulRollingRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomRsExceptMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchRestartRsExceptMetaAction;
+import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.chaos.policies.CompositeSequentialPolicy;
@@ -36,15 +38,23 @@ import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
  */
 public class ServerKillingMonkeyFactory extends MonkeyFactory {
 
+  private long gracefulRollingRestartTSSLeepTime;
+  private long rollingBatchSuspendRSSleepTime;
+  private float rollingBatchSuspendtRSRatio;
+
   @Override
   public ChaosMonkey build() {
+    loadProperties();
 
     // Destructive actions to mess things around. Cannot run batch restart
     Action[] actions1 = new Action[] {
         new RestartRandomRsExceptMetaAction(60000),
         new RestartActiveMasterAction(5000),
         new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), //only allow 2 servers to be dead
-        new ForceBalancerAction()
+      new ForceBalancerAction(),
+      new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
+      new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
+          rollingBatchSuspendtRSRatio)
     };
 
     // Action to log more info for debugging
@@ -58,4 +68,16 @@ public class ServerKillingMonkeyFactory extends MonkeyFactory {
           new PeriodicRandomActionPolicy(60 * 1000, actions1)),
       new PeriodicRandomActionPolicy(60 * 1000, actions2));
   }
+
+  private void loadProperties() {
+    gracefulRollingRestartTSSLeepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.GRACEFUL_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_GRACEFUL_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendtRSRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
+  }
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
index c97a565..f29c2a6 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.chaos.actions.DecreaseMaxHFileSizeAction;
 import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushTableAction;
+import org.apache.hadoop.hbase.chaos.actions.GracefulRollingRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.MergeRandomAdjacentRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
@@ -40,6 +41,7 @@ import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchRestartRsAction;
+import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.actions.SnapshotTableAction;
 import org.apache.hadoop.hbase.chaos.actions.SplitAllRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.SplitRandomRegionOfTableAction;
@@ -68,6 +70,9 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
   private float compactTableRatio;
   private float compactRandomRegionRatio;
   private long decreaseHFileSizeSleepTime;
+  private long gracefulRollingRestartTSSLeepTime;
+  private long rollingBatchSuspendRSSleepTime;
+  private float rollingBatchSuspendtRSRatio;
 
   @Override
   public ChaosMonkey build() {
@@ -113,6 +118,9 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
         new RestartRsHoldingMetaAction(restartRsHoldingMetaSleepTime),
         new DecreaseMaxHFileSizeAction(decreaseHFileSizeSleepTime, tableName),
         new SplitAllRegionOfTableAction(tableName),
+      new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
+      new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
+          rollingBatchSuspendtRSRatio)
     };
 
     // Action to log more info for debugging
@@ -182,5 +190,14 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
     decreaseHFileSizeSleepTime = Long.parseLong(this.properties.getProperty(
         MonkeyConstants.DECREASE_HFILE_SIZE_SLEEP_TIME,
         MonkeyConstants.DEFAULT_DECREASE_HFILE_SIZE_SLEEP_TIME + ""));
+    gracefulRollingRestartTSSLeepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.GRACEFUL_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_GRACEFUL_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendtRSRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
   }
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
index c150135..4e304fb 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/StressAssignmentManagerMonkeyFactory.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.chaos.actions.DecreaseMaxHFileSizeAction;
 import org.apache.hadoop.hbase.chaos.actions.DumpClusterStatusAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.FlushTableAction;
+import org.apache.hadoop.hbase.chaos.actions.GracefulRollingRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.MergeRandomAdjacentRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
@@ -34,6 +35,7 @@ import org.apache.hadoop.hbase.chaos.actions.RemoveColumnAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRandomRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchRestartRsAction;
+import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.actions.SplitAllRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.SplitRandomRegionOfTableAction;
 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
@@ -43,8 +45,14 @@ import org.apache.hadoop.hbase.chaos.policies.DoActionsOncePolicy;
 import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
 
 public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
+
+  private long gracefulRollingRestartTSSLeepTime;
+  private long rollingBatchSuspendRSSleepTime;
+  private float rollingBatchSuspendtRSRatio;
+
   @Override
   public ChaosMonkey build() {
+    loadProperties();
 
     // Actions that could slow down region movement.
     // These could also get regions stuck if there are issues.
@@ -73,6 +81,9 @@ public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
         new SplitAllRegionOfTableAction(tableName),
         new DecreaseMaxHFileSizeAction(MonkeyConstants.DEFAULT_DECREASE_HFILE_SIZE_SLEEP_TIME,
             tableName),
+      new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
+      new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
+          rollingBatchSuspendtRSRatio)
     };
 
     // Action to log more info for debugging
@@ -88,4 +99,16 @@ public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
         new PeriodicRandomActionPolicy(90 * 1000, actions3)
     );
   }
+
+  private void loadProperties() {
+    gracefulRollingRestartTSSLeepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.GRACEFUL_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_GRACEFUL_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
+    rollingBatchSuspendtRSRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
+  }
 }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
index 70636dd..fed5149 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
@@ -18,8 +18,9 @@
 
 package org.apache.hadoop.hbase.chaos.monkies;
 
-import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang3.RandomUtils;
@@ -90,18 +91,13 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
 
   /** Selects and returns ceil(ratio * items.length) random items from the given array */
   public static <T> List<T> selectRandomItems(T[] items, float ratio) {
-    int remaining = (int)Math.ceil(items.length * ratio);
+    int selectedNumber = (int)Math.ceil(items.length * ratio);
 
-    List<T> selectedItems = new ArrayList<>(remaining);
+    List<T> originalItems = Arrays.asList(items);
+    Collections.shuffle(originalItems);
 
-    for (int i=0; i<items.length && remaining > 0; i++) {
-      if (RandomUtils.nextFloat() < ((float)remaining/(items.length-i))) {
-        selectedItems.add(items[i]);
-        remaining--;
-      }
-    }
-
-    return selectedItems;
+    int startIndex = RandomUtils.nextInt(0, items.length - selectedNumber);
+    return originalItems.subList(startIndex, startIndex + selectedNumber);
   }
 
   private Policy[] policies;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 425b987..23ab048 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -172,6 +172,20 @@ public abstract class HBaseCluster implements Closeable, Configurable {
       throws IOException;
 
   /**
+   * Suspend the region server
+   * @param serverName the hostname to suspend the regionserver on
+   * @throws IOException if something goes wrong
+   */
+  public abstract void suspendRegionServer(ServerName serverName) throws IOException;
+
+  /**
+   * Resume the region server
+   * @param serverName the hostname to resume the regionserver on
+   * @throws IOException if something goes wrong
+   */
+  public abstract void resumeRegionServer(ServerName serverName) throws IOException;
+
+  /**
    * Starts a new zookeeper node on the given hostname or if this is a mini/local cluster,
    * silently logs warning message.
    * @param hostname the hostname to start the regionserver on
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 3adbbde..a891f66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -296,6 +296,16 @@ public class MiniHBaseCluster extends HBaseCluster {
   }
 
   @Override
+  public void suspendRegionServer(ServerName serverName) throws IOException {
+    suspendRegionServer(getRegionServerIndex(serverName));
+  }
+
+  @Override
+  public void resumeRegionServer(ServerName serverName) throws IOException {
+    resumeRegionServer(getRegionServerIndex(serverName));
+  }
+
+  @Override
   public void waitForRegionServerToStop(ServerName serverName, long timeout) throws IOException {
     //ignore timeout for now
     waitOnRegionServer(getRegionServerIndex(serverName));
@@ -494,6 +504,32 @@ public class MiniHBaseCluster extends HBaseCluster {
   }
 
   /**
+   * Suspend the specified region server
+   * @param serverNumber Used as index into a list.
+   * @return
+   */
+  public JVMClusterUtil.RegionServerThread suspendRegionServer(int serverNumber) {
+    JVMClusterUtil.RegionServerThread server =
+        hbaseCluster.getRegionServers().get(serverNumber);
+    LOG.info("Suspending {}", server.toString());
+    server.suspend();
+    return server;
+  }
+
+  /**
+   * Resume the specified region server
+   * @param serverNumber Used as index into a list.
+   * @return
+   */
+  public JVMClusterUtil.RegionServerThread resumeRegionServer(int serverNumber) {
+    JVMClusterUtil.RegionServerThread server =
+        hbaseCluster.getRegionServers().get(serverNumber);
+    LOG.info("Resuming {}", server.toString());
+    server.resume();
+    return server;
+  }
+
+  /**
    * Wait for the specified region server to stop. Removes this thread from list
    * of running threads.
    * @param serverNumber