You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/03/10 18:51:37 UTC

[hbase] branch branch-2 updated: Revert "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.

ndimiduk 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 4f76e24  Revert "HBASE-23891: Add an option to Actions to filter out meta RS (#1217)"
4f76e24 is described below

commit 4f76e24755bddeacce71a0dfd3423be7b93eca9a
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Tue Mar 10 11:48:12 2020 -0700

    Revert "HBASE-23891: Add an option to Actions to filter out meta RS (#1217)"
    
    This reverts commit 7d8fa5c818f93dffd51a2749fbf8f94c0d411b62.
---
 .../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, 84 insertions(+), 98 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 f8ba87a..903cf42 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.
    */
-  public static void loadMonkeyProperties(Properties props, Configuration conf) {
+  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 c4f87ac..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
@@ -25,7 +25,6 @@ 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;
@@ -35,13 +34,11 @@ 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;
@@ -93,7 +90,6 @@ public class Action {
   protected HBaseCluster cluster;
   protected ClusterMetrics initialStatus;
   protected ServerName[] initialServers;
-  protected Properties monkeyProps;
 
   protected long killMasterTimeout;
   protected long startMasterTimeout;
@@ -105,7 +101,6 @@ 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;
@@ -114,34 +109,25 @@ public class Action {
     Collection<ServerName> regionServers = initialStatus.getLiveServerMetrics().keySet();
     initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
 
-    monkeyProps = context.getMonkeyProps();
-    if (monkeyProps == null){
-      monkeyProps = new Properties();
-      IntegrationTestBase.loadMonkeyProperties(monkeyProps, cluster.getConf());
-    }
+    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);
 
-    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 { }
@@ -161,12 +147,6 @@ 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()]);
   }
 
@@ -378,21 +358,11 @@ 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 c76fcb1..53f9520 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,12 +49,6 @@ 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 e69de29..f03b8ec 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
@@ -0,0 +1,43 @@
+/**
+ * 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 069b604..a06a977 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 fcf2605..3091fa5 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,9 +73,8 @@ public class DistributedIssuesMonkeyFactory extends MonkeyFactory {
     // Action to log more info for debugging
     Action[] actions2 = new Action[] {new DumpClusterStatusAction()};
 
-    return new PolicyBasedChaosMonkey(properties, util,
-      new PeriodicRandomActionPolicy(action1Period, actions1),
-      new PeriodicRandomActionPolicy(action2Period, actions2));
+    return new PolicyBasedChaosMonkey(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 7f81523..52dec3b 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 400e59f..6399378 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 441695a..b875033 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 6b42899..7fbdd1f 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,7 +48,6 @@ 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";
@@ -68,7 +67,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.", "skip"));
+          "decrease.", "decrease.", "graceful.", "cpu.", "network.", "fill.", "data."));
 
   long DEFAULT_PERIODIC_ACTION1_PERIOD = 60 * 1000;
   long DEFAULT_PERIODIC_ACTION2_PERIOD = 90 * 1000;
@@ -95,7 +94,6 @@ 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 efe0fe6..def9df0 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 676b04d..5cb2d7f 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.RollingBatchRestartRsAction;
+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;
@@ -52,8 +52,7 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
     Action[] actions1 = new Action[]{
       new RestartRandomRsExceptMetaAction(60000),
       new RestartActiveMasterAction(5000),
-      new RollingBatchRestartRsAction(5000, 1.0f, 2,
-        true), // only allow 2 servers to be dead.
+      new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), // only allow 2 servers to be dead.
       new ForceBalancerAction(),
       new RestartRandomDataNodeAction(60000),
       new RestartRandomZKNodeAction(60000),
@@ -67,7 +66,7 @@ public class ServerAndDependenciesKillingMonkeyFactory extends MonkeyFactory {
       new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(properties, util,
+    return new PolicyBasedChaosMonkey(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 6a5efed..3f2edcc 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.RollingBatchRestartRsAction;
+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;
@@ -50,8 +50,7 @@ public class ServerKillingMonkeyFactory extends MonkeyFactory {
     Action[] actions1 = new Action[] {
         new RestartRandomRsExceptMetaAction(60000),
         new RestartActiveMasterAction(5000),
-      new RollingBatchRestartRsAction(5000, 1.0f, 2,
-          true), //only allow 2 servers to be dead
+        new RollingBatchRestartRsExceptMetaAction(5000, 1.0f, 2), //only allow 2 servers to be dead
       new ForceBalancerAction(),
       new GracefulRollingRestartRsAction(gracefulRollingRestartTSSLeepTime),
       new RollingBatchSuspendResumeRsAction(rollingBatchSuspendRSSleepTime,
@@ -63,7 +62,7 @@ public class ServerKillingMonkeyFactory extends MonkeyFactory {
         new DumpClusterStatusAction()
     };
 
-    return new PolicyBasedChaosMonkey(properties, util,
+    return new PolicyBasedChaosMonkey(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 3bba74d..cbd492e 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 16d6b29..b25bef7 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(properties, util,
+    return new PolicyBasedChaosMonkey(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 f9c8184..28ab7a0 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(properties, util, chaosPolicy);
+    return new PolicyBasedChaosMonkey(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 dc2ac13..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
@@ -22,7 +22,6 @@ 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;
@@ -44,7 +43,6 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
   public static final long TIMEOUT = ONE_MIN;
 
   final IntegrationTestingUtility util;
-  final Properties monkeyProps;
 
   /**
    * Construct a new ChaosMonkey
@@ -52,23 +50,11 @@ 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(Properties monkeyProps, IntegrationTestingUtility util,
-    Collection<Policy> policies) {
-    this.monkeyProps = monkeyProps;
+  public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Collection<Policy> policies) {
     this.util = util;
     this.policies = policies.toArray(new Policy[policies.size()]);
   }
@@ -120,9 +106,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(context);
+      policies[i].init(new Policy.PolicyContext(this.util));
       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 ad68b66..81267a6 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,8 +18,6 @@
 
 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;
@@ -49,8 +47,8 @@ public abstract class Policy extends StoppableImplementation implements Runnable
 
     Policy policy = null;
 
-    public PolicyContext(Properties monkeyProps, IntegrationTestingUtility util) {
-      super(monkeyProps, util);
+    public PolicyContext(IntegrationTestingUtility util) {
+      super(util);
     }
 
     @Override