You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by wc...@apache.org on 2020/03/06 11:24:02 UTC

[hbase] branch branch-2 updated: HBASE-23891: Add an option to Actions to filter out meta RS (#1217)

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

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


The following commit(s) were added to refs/heads/branch-2 by this push:
     new 7d8fa5c  HBASE-23891: Add an option to Actions to filter out meta RS (#1217)
7d8fa5c is described below

commit 7d8fa5c818f93dffd51a2749fbf8f94c0d411b62
Author: BukrosSzabolcs <bu...@gmail.com>
AuthorDate: Fri Mar 6 11:44:39 2020 +0100

    HBASE-23891: Add an option to Actions to filter out meta RS (#1217)
    
    Signed-off-by: Wellington Chevreuil <wc...@apache.org>
    (cherry picked from commit 4cb60327be29a4d1b46f4969186a3136726cb557)
---
 .../apache/hadoop/hbase/IntegrationTestBase.java   |  2 +-
 .../apache/hadoop/hbase/chaos/actions/Action.java  | 66 ++++++++++++++++------
 .../chaos/actions/RollingBatchRestartRsAction.java |  6 ++
 .../RollingBatchRestartRsExceptMetaAction.java     | 43 --------------
 .../chaos/factories/DataIssuesMonkeyFactory.java   |  2 +-
 .../factories/DistributedIssuesMonkeyFactory.java  |  5 +-
 .../factories/MasterKillingMonkeyFactory.java      |  2 +-
 .../chaos/factories/MobNoKillMonkeyFactory.java    |  2 +-
 .../MobSlowDeterministicMonkeyFactory.java         |  2 +-
 .../hbase/chaos/factories/MonkeyConstants.java     |  4 +-
 .../hbase/chaos/factories/NoKillMonkeyFactory.java |  2 +-
 .../ServerAndDependenciesKillingMonkeyFactory.java |  7 ++-
 .../factories/ServerKillingMonkeyFactory.java      |  7 ++-
 .../factories/SlowDeterministicMonkeyFactory.java  |  2 +-
 .../StressAssignmentManagerMonkeyFactory.java      |  2 +-
 .../chaos/factories/UnbalanceMonkeyFactory.java    |  2 +-
 .../chaos/monkies/PolicyBasedChaosMonkey.java      | 20 ++++++-
 .../apache/hadoop/hbase/chaos/policies/Policy.java |  6 +-
 18 files changed, 98 insertions(+), 84 deletions(-)

diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBase.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBase.java
index 903cf42..f8ba87a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBase.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBase.java
@@ -111,7 +111,7 @@ public abstract class IntegrationTestBase extends AbstractHBaseTool {
    * Loads entries from the provided {@code conf} into {@code props} when the configuration key
    * is one that may be configuring ChaosMonkey actions.
    */
-  void loadMonkeyProperties(Properties props, Configuration conf) {
+  public static void loadMonkeyProperties(Properties props, Configuration conf) {
     for (Entry<String,String> entry : conf) {
       for (String prefix : MonkeyConstants.MONKEY_CONFIGURATION_KEY_PREFIXES) {
         if (entry.getKey().startsWith(prefix)) {
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 be4251a..c4f87ac 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
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 import java.util.function.BiConsumer;
 import java.util.function.Consumer;
@@ -34,11 +35,13 @@ import org.apache.hadoop.hbase.ClusterMetrics;
 import org.apache.hadoop.hbase.HBaseCluster;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.IntegrationTestBase;
 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;
+import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
@@ -90,6 +93,7 @@ public class Action {
   protected HBaseCluster cluster;
   protected ClusterMetrics initialStatus;
   protected ServerName[] initialServers;
+  protected Properties monkeyProps;
 
   protected long killMasterTimeout;
   protected long startMasterTimeout;
@@ -101,6 +105,7 @@ public class Action {
   protected long startDataNodeTimeout;
   protected long killNameNodeTimeout;
   protected long startNameNodeTimeout;
+  protected boolean skipMetaRS;
 
   public void init(ActionContext context) throws IOException {
     this.context = context;
@@ -109,25 +114,34 @@ public class Action {
     Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
     initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
 
-    killMasterTimeout = cluster.getConf().getLong(KILL_MASTER_TIMEOUT_KEY,
-      KILL_MASTER_TIMEOUT_DEFAULT);
-    startMasterTimeout = cluster.getConf().getLong(START_MASTER_TIMEOUT_KEY,
-      START_MASTER_TIMEOUT_DEFAULT);
-    killRsTimeout = cluster.getConf().getLong(KILL_RS_TIMEOUT_KEY, KILL_RS_TIMEOUT_DEFAULT);
-    startRsTimeout = cluster.getConf().getLong(START_RS_TIMEOUT_KEY, START_RS_TIMEOUT_DEFAULT);
-    killZkNodeTimeout = cluster.getConf().getLong(KILL_ZK_NODE_TIMEOUT_KEY,
-      KILL_ZK_NODE_TIMEOUT_DEFAULT);
-    startZkNodeTimeout = cluster.getConf().getLong(START_ZK_NODE_TIMEOUT_KEY,
-      START_ZK_NODE_TIMEOUT_DEFAULT);
-    killDataNodeTimeout = cluster.getConf().getLong(KILL_DATANODE_TIMEOUT_KEY,
-      KILL_DATANODE_TIMEOUT_DEFAULT);
-    startDataNodeTimeout = cluster.getConf().getLong(START_DATANODE_TIMEOUT_KEY,
-      START_DATANODE_TIMEOUT_DEFAULT);
-    killNameNodeTimeout =
-        cluster.getConf().getLong(KILL_NAMENODE_TIMEOUT_KEY, KILL_NAMENODE_TIMEOUT_DEFAULT);
-    startNameNodeTimeout =
-        cluster.getConf().getLong(START_NAMENODE_TIMEOUT_KEY, START_NAMENODE_TIMEOUT_DEFAULT);
+    monkeyProps = context.getMonkeyProps();
+    if (monkeyProps == null){
+      monkeyProps = new Properties();
+      IntegrationTestBase.loadMonkeyProperties(monkeyProps, cluster.getConf());
+    }
 
+    killMasterTimeout = Long.parseLong(monkeyProps.getProperty(
+      KILL_MASTER_TIMEOUT_KEY, KILL_MASTER_TIMEOUT_DEFAULT + ""));
+    startMasterTimeout = Long.parseLong(monkeyProps.getProperty(START_MASTER_TIMEOUT_KEY,
+      START_MASTER_TIMEOUT_DEFAULT + ""));
+    killRsTimeout = Long.parseLong(monkeyProps.getProperty(KILL_RS_TIMEOUT_KEY,
+      KILL_RS_TIMEOUT_DEFAULT + ""));
+    startRsTimeout = Long.parseLong(monkeyProps.getProperty(START_RS_TIMEOUT_KEY,
+      START_RS_TIMEOUT_DEFAULT+ ""));
+    killZkNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_ZK_NODE_TIMEOUT_KEY,
+      KILL_ZK_NODE_TIMEOUT_DEFAULT + ""));
+    startZkNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_ZK_NODE_TIMEOUT_KEY,
+      START_ZK_NODE_TIMEOUT_DEFAULT + ""));
+    killDataNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_DATANODE_TIMEOUT_KEY,
+      KILL_DATANODE_TIMEOUT_DEFAULT + ""));
+    startDataNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_DATANODE_TIMEOUT_KEY,
+      START_DATANODE_TIMEOUT_DEFAULT + ""));
+    killNameNodeTimeout = Long.parseLong(monkeyProps.getProperty(KILL_NAMENODE_TIMEOUT_KEY,
+      KILL_NAMENODE_TIMEOUT_DEFAULT + ""));
+    startNameNodeTimeout = Long.parseLong(monkeyProps.getProperty(START_NAMENODE_TIMEOUT_KEY,
+      START_NAMENODE_TIMEOUT_DEFAULT + ""));
+    skipMetaRS = Boolean.parseBoolean(monkeyProps.getProperty(MonkeyConstants.SKIP_META_RS,
+      MonkeyConstants.DEFAULT_SKIP_META_RS + ""));
   }
 
   public void perform() throws Exception { }
@@ -147,6 +161,12 @@ public class Action {
     ArrayList<ServerName> tmp = new ArrayList<>(count);
     tmp.addAll(regionServers);
     tmp.removeAll(masters);
+
+    if(skipMetaRS){
+      ServerName metaServer = cluster.getServerHoldingMeta();
+      tmp.remove(metaServer);
+    }
+
     return tmp.toArray(new ServerName[tmp.size()]);
   }
 
@@ -358,11 +378,21 @@ public class Action {
    */
   public static class ActionContext {
     private IntegrationTestingUtility util;
+    private Properties monkeyProps = null;
 
     public ActionContext(IntegrationTestingUtility util) {
       this.util = util;
     }
 
+    public ActionContext(Properties monkeyProps, IntegrationTestingUtility util) {
+      this.util = util;
+      this.monkeyProps = monkeyProps;
+    }
+
+    public Properties getMonkeyProps(){
+      return monkeyProps;
+    }
+
     public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
       return util;
     }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
index 53f9520..c76fcb1 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsAction.java
@@ -49,6 +49,12 @@ public class RollingBatchRestartRsAction extends BatchRestartRsAction {
     this.maxDeadServers = maxDeadServers;
   }
 
+  public RollingBatchRestartRsAction(long sleepTime, float ratio, int maxDeadServers,
+    boolean skipMetaRS) {
+    this(sleepTime, ratio, maxDeadServers);
+    this.skipMetaRS = skipMetaRS;
+  }
+
   enum KillOrStart {
     KILL,
     START
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsExceptMetaAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsExceptMetaAction.java
index f03b8ec..e69de29 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsExceptMetaAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/RollingBatchRestartRsExceptMetaAction.java
@@ -1,43 +0,0 @@
-/**
- * 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.util.List;
-
-import org.apache.hadoop.hbase.ServerName;
-
-/**
- * Same as in {@link RollingBatchRestartRsAction} except that this action
- * does not restart the region server holding the META table.
- */
-public class RollingBatchRestartRsExceptMetaAction extends RollingBatchRestartRsAction {
-
-  public RollingBatchRestartRsExceptMetaAction(long sleepTime, float ratio, int maxDeadServers) {
-    super(sleepTime, ratio, maxDeadServers);
-  }
-
-  @Override
-  protected List<ServerName> selectServers() throws java.io.IOException {
-    ServerName metaServer = cluster.getServerHoldingMeta();
-    List<ServerName> servers = super.selectServers();
-    servers.remove(metaServer);
-    return servers;
-  };
-
-}
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DataIssuesMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DataIssuesMonkeyFactory.java
index a06a977..069b604 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DataIssuesMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DataIssuesMonkeyFactory.java
@@ -53,7 +53,7 @@ public class DataIssuesMonkeyFactory extends MonkeyFactory {
       new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
       new PeriodicRandomActionPolicy(action1Period, actions1),
       new PeriodicRandomActionPolicy(action2Period, actions2));
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DistributedIssuesMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DistributedIssuesMonkeyFactory.java
index 3091fa5..fcf2605 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DistributedIssuesMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/DistributedIssuesMonkeyFactory.java
@@ -73,8 +73,9 @@ public class DistributedIssuesMonkeyFactory extends MonkeyFactory {
     // Action to log more info for debugging
     Action[] actions2 = new Action[] {new DumpClusterStatusAction()};
 
-    return new PolicyBasedChaosMonkey(util, new PeriodicRandomActionPolicy(action1Period, actions1),
-        new PeriodicRandomActionPolicy(action2Period, actions2));
+    return new PolicyBasedChaosMonkey(properties, util,
+      new PeriodicRandomActionPolicy(action1Period, actions1),
+      new PeriodicRandomActionPolicy(action2Period, actions2));
   }
 
   private void loadProperties() {
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MasterKillingMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MasterKillingMonkeyFactory.java
index 52dec3b..7f81523 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MasterKillingMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MasterKillingMonkeyFactory.java
@@ -50,7 +50,7 @@ public class MasterKillingMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
         new PeriodicRandomActionPolicy(action1Period, actions1),
         new PeriodicRandomActionPolicy(action2Period, actions2));
   }
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobNoKillMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobNoKillMonkeyFactory.java
index 6399378..400e59f 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobNoKillMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobNoKillMonkeyFactory.java
@@ -75,7 +75,7 @@ public class MobNoKillMonkeyFactory extends MonkeyFactory {
 
     Action[] actions4 = new Action[] { new DumpClusterStatusAction() };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
       new TwoConcurrentActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD, actions1,
         actions2),
       new PeriodicRandomActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION2_PERIOD,actions3),
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobSlowDeterministicMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobSlowDeterministicMonkeyFactory.java
index b875033..441695a 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobSlowDeterministicMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MobSlowDeterministicMonkeyFactory.java
@@ -118,7 +118,7 @@ public class MobSlowDeterministicMonkeyFactory extends MonkeyFactory {
             new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
             new PeriodicRandomActionPolicy(action1Period, actions1),
             new PeriodicRandomActionPolicy(action2Period, actions2),
             new CompositeSequentialPolicy(
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 7fbdd1f..6b42899 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
@@ -48,6 +48,7 @@ public interface MonkeyConstants {
   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";
+  String SKIP_META_RS = "skip.meta.rs";
   String CPU_LOAD_DURATION = "cpu.load.duration";
   String CPU_LOAD_PROCESSES = "cpu.load.processes";
   String NETWORK_ISSUE_COMMAND_TIMEOUT = "network.issue.command.timeout";
@@ -67,7 +68,7 @@ public interface MonkeyConstants {
    */
   Set<String> MONKEY_CONFIGURATION_KEY_PREFIXES = new HashSet<>(
       Arrays.asList("sdm.", "move.", "restart.", "batch.", "rolling.", "compact.", "unbalance.",
-          "decrease.", "decrease.", "graceful.", "cpu.", "network.", "fill.", "data."));
+          "decrease.", "decrease.", "graceful.", "cpu.", "network.", "fill.", "data.", "skip"));
 
   long DEFAULT_PERIODIC_ACTION1_PERIOD = 60 * 1000;
   long DEFAULT_PERIODIC_ACTION2_PERIOD = 90 * 1000;
@@ -94,6 +95,7 @@ public interface MonkeyConstants {
   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;
+  boolean DEFAULT_SKIP_META_RS = false;
   long DEFAULT_CPU_LOAD_DURATION = 5 * 60 * 1000;
   long DEFAULT_CPU_LOAD_PROCESSES = 2;
   long DEFAULT_NETWORK_ISSUE_COMMAND_TIMEOUT = 30 * 1000;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
index def9df0..efe0fe6 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
@@ -79,7 +79,7 @@ public class NoKillMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
         new TwoConcurrentActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD, actions1, actions2),
         new PeriodicRandomActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION2_PERIOD,actions3),
         new PeriodicRandomActionPolicy(MonkeyConstants.DEFAULT_PERIODIC_ACTION4_PERIOD,actions4));
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 5cb2d7f..676b04d 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
@@ -26,7 +26,7 @@ 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.RollingBatchRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -52,7 +52,8 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
     Action[] actions1 = new Action[]{
       new RestartRandomRsExceptMetaAction(60000),
       new RestartActiveMasterAction(5000),
-      new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), // only allow 2 servers to be dead.
+      new RollingBatchRestartRsAction(5000, 1.0f, 2,
+        true), // only allow 2 servers to be dead.
       new ForceBalancerAction(),
       new RestartRandomDataNodeAction(60000),
       new RestartRandomZKNodeAction(60000),
@@ -66,7 +67,7 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
       new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
       new CompositeSequentialPolicy(
         new DoActionsOncePolicy(60 * 1000, actions1),
         new PeriodicRandomActionPolicy(60 * 1000, actions1)),
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 3f2edcc..6a5efed 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
@@ -24,7 +24,7 @@ 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.RollingBatchRestartRsAction;
 import org.apache.hadoop.hbase.chaos.actions.RollingBatchSuspendResumeRsAction;
 import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
 import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
@@ -50,7 +50,8 @@ public class ServerKillingMonkeyFactory extends MonkeyFactory {
     Action[] actions1 = new Action[] {
         new RestartRandomRsExceptMetaAction(60000),
         new RestartActiveMasterAction(5000),
-        new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), //only allow 2 servers to be dead
+      new RollingBatchRestartRsAction(5000, 1.0f, 2,
+          true), //only allow 2 servers to be dead
       new ForceBalancerAction(),
       new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
       new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
@@ -62,7 +63,7 @@ public class ServerKillingMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
       new CompositeSequentialPolicy(
           new DoActionsOncePolicy(60 * 1000, actions1),
           new PeriodicRandomActionPolicy(60 * 1000, actions1)),
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 cbd492e..3bba74d 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
@@ -128,7 +128,7 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
         new PeriodicRandomActionPolicy(action1Period, actions1),
         new PeriodicRandomActionPolicy(action2Period, actions2),
         new CompositeSequentialPolicy(
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 b25bef7..16d6b29 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
@@ -91,7 +91,7 @@ public class StressAssignmentManagerMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(util,
+    return new PolicyBasedChaosMonkey(properties, util,
         new PeriodicRandomActionPolicy(90 * 1000, actions1),
         new CompositeSequentialPolicy(
             new DoActionsOncePolicy(90 * 1000, actions2),
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/UnbalanceMonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/UnbalanceMonkeyFactory.java
index 28ab7a0..f9c8184 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/UnbalanceMonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/UnbalanceMonkeyFactory.java
@@ -40,7 +40,7 @@ public class UnbalanceMonkeyFactory extends MonkeyFactory {
         new UnbalanceKillAndRebalanceAction(waitForUnbalanceMilliSec, waitForKillMilliSec,
             waitAfterBalanceMilliSec, killMetaRs));
 
-    return new PolicyBasedChaosMonkey(util, chaosPolicy);
+    return new PolicyBasedChaosMonkey(properties, util, chaosPolicy);
   }
 
   private void loadProperties() {
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 fed5149..dc2ac13 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
@@ -22,6 +22,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.Properties;
 
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
@@ -43,6 +44,7 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
   public static final long TIMEOUT = ONE_MIN;
 
   final IntegrationTestingUtility util;
+  final Properties monkeyProps;
 
   /**
    * Construct a new ChaosMonkey
@@ -50,11 +52,23 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
    * @param policies custom policies to use
    */
   public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Policy... policies) {
+    this(null, util, policies);
+  }
+
+  public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Collection<Policy> policies) {
+    this(null, util, policies);
+  }
+
+  public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
+    Policy... policies) {
+    this.monkeyProps = monkeyProps;
     this.util = util;
     this.policies = policies;
   }
 
-  public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Collection<Policy> policies) {
+  public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
+    Collection<Policy> policies) {
+    this.monkeyProps = monkeyProps;
     this.util = util;
     this.policies = policies.toArray(new Policy[policies.size()]);
   }
@@ -106,9 +120,9 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
   @Override
   public void start() throws Exception {
     monkeyThreads = new Thread[policies.length];
-
+    Policy.PolicyContext context = new Policy.PolicyContext(monkeyProps, this.util);
     for (int i=0; i<policies.length; i++) {
-      policies[i].init(new Policy.PolicyContext(this.util));
+      policies[i].init(context);
       Thread monkeyThread = new Thread(policies[i], "ChaosMonkey");
       monkeyThread.start();
       monkeyThreads[i] = monkeyThread;
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/Policy.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/Policy.java
index 81267a6..ad68b66 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/Policy.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/Policy.java
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hbase.chaos.policies;
 
+import java.util.Properties;
+
 import org.apache.hadoop.hbase.IntegrationTestingUtility;
 import org.apache.hadoop.hbase.chaos.actions.Action;
 import org.apache.hadoop.hbase.util.StoppableImplementation;
@@ -47,8 +49,8 @@ public abstract class Policy extends StoppableImplementation implements Runnable
 
     Policy policy = null;
 
-    public PolicyContext(IntegrationTestingUtility util) {
-      super(util);
+    public PolicyContext(Properties monkeyProps, IntegrationTestingUtility util) {
+      super(monkeyProps, util);
     }
 
     @Override