You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2014/06/19 23:51:18 UTC

[1/2] git commit: HBASE-10289 Avoid random port usage by default JMX Server. Create Custome JMX server (Qiang Tian). DOC ADDENDUM

Repository: hbase
Updated Branches:
  refs/heads/master 45bc13d87 -> 5764df297


HBASE-10289 Avoid random port usage by default JMX Server. Create Custome JMX server (Qiang Tian).  DOC ADDENDUM


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/b16e36a5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/b16e36a5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/b16e36a5

Branch: refs/heads/master
Commit: b16e36a5b2650b2ce2c7686c6653c77074481115
Parents: 45bc13d
Author: Michael Stack <st...@apache.org>
Authored: Tue Jun 17 22:42:13 2014 -0500
Committer: Michael Stack <st...@apache.org>
Committed: Thu Jun 19 14:38:19 2014 -0700

----------------------------------------------------------------------
 src/main/docbkx/configuration.xml | 14 +++++++++++++-
 1 file changed, 13 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b16e36a5/src/main/docbkx/configuration.xml
----------------------------------------------------------------------
diff --git a/src/main/docbkx/configuration.xml b/src/main/docbkx/configuration.xml
index 1777b8d..00effab 100644
--- a/src/main/docbkx/configuration.xml
+++ b/src/main/docbkx/configuration.xml
@@ -1474,7 +1474,8 @@ index e70ebc6..96f8c27 100644
           for details)
         </para>
         <para>As an alternative, You can use the coprocessor-based JMX implementation provided
-          by HBase. To enable it, add below property in <filename>hbase-site.xml</filename>:
+          by HBase. To enable it in 0.99 or above, add below property in
+          <filename>hbase-site.xml</filename>:
         <programlisting><![CDATA[
 <property>
     <name>hbase.coprocessor.regionserver.classes</name>
@@ -1549,6 +1550,17 @@ export HBASE_REGIONSERVER_OPTS="$HBASE_REGIONSERVER_OPTS $HBASE_JMX_BASE "
 jconsole -J-Djavax.net.ssl.trustStore=/home/tianq/jconsoleKeyStore
       </screen>
         </para>
+        <para>NOTE: for HBase 0.98, To enable the HBase JMX implementation on Master, you also
+          need to add below property in <filename>hbase-site.xml</filename>:
+        <programlisting><![CDATA[
+<property>
+    <name>hbase.coprocessor.master.classes</name>
+    <value>org.apache.hadoop.hbase.JMXListener</value>
+</property>
+]]></programlisting>
+          The corresponding properties for port configuration are master.rmi.registry.port
+          (by default 10101) and master.rmi.connector.port(by default the same as registry.port)
+        </para>
     </section>
 
    </section>


[2/2] git commit: HBASE-11348 Make frequency and sleep times of chaos monkeys configurable (Vandan Ayyalasomayajula)

Posted by st...@apache.org.
HBASE-11348 Make frequency and sleep times of chaos monkeys configurable (Vandan Ayyalasomayajula)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5764df29
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5764df29
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5764df29

Branch: refs/heads/master
Commit: 5764df2974e68efd69d581478618cffe1395e547
Parents: b16e36a
Author: Michael Stack <st...@apache.org>
Authored: Thu Jun 19 14:50:50 2014 -0700
Committer: Michael Stack <st...@apache.org>
Committed: Thu Jun 19 14:50:50 2014 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/IntegrationTestBase.java       | 27 ++++++
 .../chaos/actions/MoveRegionsOfTableAction.java |  7 +-
 .../UnbalanceKillAndRebalanceAction.java        | 19 ++--
 .../hbase/chaos/factories/MonkeyConstants.java  | 64 +++++++++++++
 .../hbase/chaos/factories/MonkeyFactory.java    |  9 ++
 .../SlowDeterministicMonkeyFactory.java         | 99 +++++++++++++++++---
 .../chaos/factories/UnbalanceMonkeyFactory.java | 28 +++++-
 .../hadoop/hbase/mttr/IntegrationTestMTTR.java  |  4 +-
 src/main/docbkx/developer.xml                   | 21 +++++
 9 files changed, 249 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/IntegrationTestBase.java
----------------------------------------------------------------------
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 2637f8b..77bdd68 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
@@ -18,9 +18,12 @@
 
 package org.apache.hadoop.hbase;
 
+import java.io.IOException;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.commons.cli.CommandLine;
+import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -32,15 +35,23 @@ import org.junit.Before;
 
 /**
  * Base class for HBase integration tests that want to use the Chaos Monkey.
+ * Usage: bin/hbase <sub_class_of_IntegrationTestBase> <options>
+ * Options: -h,--help Show usage
+ *          -m,--monkey <arg> Which chaos monkey to run
+ *          -monkeyProps <arg> The properties file for specifying chaos monkey properties.
+ *          -ncc Option to not clean up the cluster at the end.
  */
 public abstract class IntegrationTestBase extends AbstractHBaseTool {
+
   public static final String NO_CLUSTER_CLEANUP_LONG_OPT = "noClusterCleanUp";
   public static final String MONKEY_LONG_OPT = "monkey";
+  public static final String CHAOS_MONKEY_PROPS = "monkeyProps";
   private static final Log LOG = LogFactory.getLog(IntegrationTestBase.class);
 
   protected IntegrationTestingUtility util;
   protected ChaosMonkey monkey;
   protected String monkeyToUse;
+  protected Properties monkeyProps;
   protected boolean noClusterCleanUp = false;
 
   public IntegrationTestBase() {
@@ -56,6 +67,8 @@ public abstract class IntegrationTestBase extends AbstractHBaseTool {
     addOptWithArg("m", MONKEY_LONG_OPT, "Which chaos monkey to run");
     addOptNoArg("ncc", NO_CLUSTER_CLEANUP_LONG_OPT,
       "Don't clean up the cluster at the end");
+    addOptWithArg(CHAOS_MONKEY_PROPS, "The properties file for specifying chaos "
+        + "monkey properties.");
   }
 
   @Override
@@ -66,6 +79,19 @@ public abstract class IntegrationTestBase extends AbstractHBaseTool {
     if (cmd.hasOption(NO_CLUSTER_CLEANUP_LONG_OPT)) {
       noClusterCleanUp = true;
     }
+    monkeyProps = new Properties();
+    if (cmd.hasOption(CHAOS_MONKEY_PROPS)) {
+      String chaosMonkeyPropsFile = cmd.getOptionValue(CHAOS_MONKEY_PROPS);
+      if (StringUtils.isNotEmpty(chaosMonkeyPropsFile)) {
+        try {
+          monkeyProps.load(this.getClass().getClassLoader()
+              .getResourceAsStream(chaosMonkeyPropsFile));
+        } catch (IOException e) {
+          LOG.warn(e);
+          System.exit(EXIT_FAILURE);
+        }
+      }
+    }
   }
 
   @Override
@@ -113,6 +139,7 @@ public abstract class IntegrationTestBase extends AbstractHBaseTool {
     }
     monkey = fact.setUtil(util)
                  .setTableName(getTablename())
+                 .setProperties(monkeyProps)
                  .setColumnFamilies(getColumnFamilies()).build();
     monkey.start();
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
index 4c5db1e..d40ef07 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/MoveRegionsOfTableAction.java
@@ -25,6 +25,7 @@ import java.util.List;
 import org.apache.commons.lang.math.RandomUtils;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 
@@ -38,14 +39,14 @@ public class MoveRegionsOfTableAction extends Action {
   private final long maxTime;
 
   public MoveRegionsOfTableAction(String tableName) {
-    this(-1, tableName);
+    this(-1, MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, tableName);
   }
 
-  public MoveRegionsOfTableAction(long sleepTime, String tableName) {
+  public MoveRegionsOfTableAction(long sleepTime, long maxSleepTime, String tableName) {
     this.sleepTime = sleepTime;
     this.tableNameBytes = Bytes.toBytes(tableName);
     this.tableName = tableName;
-    this.maxTime = 10 * 60 * 1000; // 10 min default
+    this.maxTime = maxSleepTime;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
index 540b67a..a97a9c4 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/actions/UnbalanceKillAndRebalanceAction.java
@@ -36,9 +36,16 @@ public class UnbalanceKillAndRebalanceAction extends Action {
   private static final double HOARD_FRC_OF_REGIONS = 0.8;
   /** Waits between calling unbalance and killing servers, kills and rebalance, and rebalance
    * and restarting the servers; to make sure these events have time to impact the cluster. */
-  private static final long WAIT_FOR_UNBALANCE_MS = 2 * 1000;
-  private static final long WAIT_FOR_KILLS_MS = 2 * 1000;
-  private static final long WAIT_AFTER_BALANCE_MS = 5 * 1000;
+  private long waitForUnbalanceMilliSec;
+  private long waitForKillsMilliSec;
+  private long waitAfterBalanceMilliSec;
+
+  public UnbalanceKillAndRebalanceAction(long waitUnbalance, long waitKill, long waitAfterBalance) {
+    super();
+    waitForUnbalanceMilliSec = waitUnbalance;
+    waitForKillsMilliSec = waitKill;
+    waitAfterBalanceMilliSec = waitAfterBalance;
+  }
 
   @Override
   public void perform() throws Exception {
@@ -53,13 +60,13 @@ public class UnbalanceKillAndRebalanceAction extends Action {
       targetServers.add(victimServers.remove(victimIx));
     }
     unbalanceRegions(status, victimServers, targetServers, HOARD_FRC_OF_REGIONS);
-    Thread.sleep(WAIT_FOR_UNBALANCE_MS);
+    Thread.sleep(waitForUnbalanceMilliSec);
     for (int i = 0; i < liveCount; ++i) {
       killRs(targetServers.get(i));
     }
-    Thread.sleep(WAIT_FOR_KILLS_MS);
+    Thread.sleep(waitForKillsMilliSec);
     forceBalancer();
-    Thread.sleep(WAIT_AFTER_BALANCE_MS);
+    Thread.sleep(waitAfterBalanceMilliSec);
     for (int i = 0; i < liveCount; ++i) {
       startRs(targetServers.get(i));
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..3333b26
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyConstants.java
@@ -0,0 +1,64 @@
+/**
+ * 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.factories;
+
+public interface MonkeyConstants {
+
+  public static final String PERIODIC_ACTION1_PERIOD = "sdm.action1.period";
+  public static final String PERIODIC_ACTION2_PERIOD = "sdm.action2.period";
+  public static final String PERIODIC_ACTION4_PERIOD = "sdm.action4.period";
+  public static final String COMPOSITE_ACTION3_PERIOD = "sdm.action3.period";
+  public static final String MOVE_REGIONS_MAX_TIME = "move.regions.max.time";
+  public static final String MOVE_REGIONS_SLEEP_TIME = "move.regions.sleep.time";
+  public static final String MOVE_RANDOM_REGION_SLEEP_TIME = "move.randomregion.sleep.time";
+  public static final String RESTART_RANDOM_RS_SLEEP_TIME = "restart.random.rs.sleep.time";
+  public static final String BATCH_RESTART_RS_SLEEP_TIME = "batch.restart.rs.sleep.time";
+  public static final String BATCH_RESTART_RS_RATIO = "batch.restart.rs.ratio";
+  public static final String RESTART_ACTIVE_MASTER_SLEEP_TIME = "restart.active.master.sleep.time";
+  public static final String ROLLING_BATCH_RESTART_RS_SLEEP_TIME = "rolling.batch.restart.rs.sleep.time";
+  public static final String ROLLING_BATCH_RESTART_RS_RATIO = "rolling.batch.restart.rs.ratio";
+  public static final String RESTART_RS_HOLDING_META_SLEEP_TIME = "restart.rs.holding.meta.sleep.time";
+  public static final String COMPACT_TABLE_ACTION_RATIO = "compact.table.ratio";
+  public static final String COMPACT_RANDOM_REGION_RATIO = "compact.random.region.ratio";
+  public static final String UNBALANCE_CHAOS_EVERY_MS = "unbalance.chaos.period";
+  public static final String UNBALANCE_WAIT_FOR_UNBALANCE_MS = "unbalance.action.wait.period";
+  public static final String UNBALANCE_WAIT_FOR_KILLS_MS = "unbalance.action.kill.period";
+  public static final String UNBALANCE_WAIT_AFTER_BALANCE_MS = "unbalance.action.wait.after.period";
+
+  public static final long DEFAULT_PERIODIC_ACTION1_PERIOD = 60 * 1000;
+  public static final long DEFAULT_PERIODIC_ACTION2_PERIOD = 90 * 1000;
+  public static final long DEFAULT_PERIODIC_ACTION4_PERIOD = 90 * 1000;
+  public static final long DEFAULT_COMPOSITE_ACTION3_PERIOD = 150 * 1000;
+  public static final long DEFAULT_MOVE_REGIONS_MAX_TIME = 10 * 60 * 1000;
+  public static final long DEFAULT_MOVE_REGIONS_SLEEP_TIME = 800;
+  public static final long DEFAULT_MOVE_RANDOM_REGION_SLEEP_TIME = 800;
+  public static final long DEFAULT_RESTART_RANDOM_RS_SLEEP_TIME = 60000;
+  public static final long DEFAULT_BATCH_RESTART_RS_SLEEP_TIME = 5000;
+  public static final float DEFAULT_BATCH_RESTART_RS_RATIO = 0.5f;
+  public static final long DEFAULT_RESTART_ACTIVE_MASTER_SLEEP_TIME = 5000;
+  public static final long DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME = 5000;
+  public static final float DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO = 1.0f;
+  public static final long DEFAULT_RESTART_RS_HOLDING_META_SLEEP_TIME = 35000;
+  public static final float DEFAULT_COMPACT_TABLE_ACTION_RATIO = 0.5f;
+  public static final float DEFAULT_COMPACT_RANDOM_REGION_RATIO = 0.6f;
+  public static final long DEFAULT_UNBALANCE_CHAOS_EVERY_MS = 65 * 1000;
+  public static final long DEFAULT_UNBALANCE_WAIT_FOR_UNBALANCE_MS = 2 * 1000;
+  public static final long DEFAULT_UNBALANCE_WAIT_FOR_KILLS_MS = 2 * 1000;
+  public static final long DEFAULT_UNBALANCE_WAIT_AFTER_BALANCE_MS = 5 * 1000;
+
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyFactory.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyFactory.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyFactory.java
index 9518689..4f3824b 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyFactory.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/MonkeyFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hbase.chaos.factories;
 
 import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
 import com.google.common.collect.ImmutableMap;
@@ -33,6 +34,7 @@ public abstract class MonkeyFactory {
   protected String tableName;
   protected Set<String> columnFamilies;
   protected IntegrationTestingUtility util;
+  protected Properties properties = new Properties();
 
   public MonkeyFactory setTableName(String tableName) {
     this.tableName = tableName;
@@ -49,6 +51,13 @@ public abstract class MonkeyFactory {
     return this;
   }
 
+  public MonkeyFactory setProperties(Properties props) {
+    if (props != null) {
+      this.properties = props;
+    }
+    return this;
+  }
+
   public abstract ChaosMonkey build();
 
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/SlowDeterministicMonkeyFactory.java
----------------------------------------------------------------------
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 2b7ea20..70d64b9 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
@@ -45,17 +45,37 @@ import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
 import org.apache.hadoop.hbase.chaos.policies.CompositeSequentialPolicy;
 import org.apache.hadoop.hbase.chaos.policies.DoActionsOncePolicy;
 import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
+import org.junit.Assert;
 
 public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
+
+  private long action1Period;
+  private long action2Period;
+  private long action3Period;
+  private long action4Period;
+  private long moveRegionsMaxTime;
+  private long moveRegionsSleepTime;
+  private long moveRandomRegionSleepTime;
+  private long restartRandomRSSleepTime;
+  private long batchRestartRSSleepTime;
+  private float batchRestartRSRatio;
+  private long restartActiveMasterSleepTime;
+  private long rollingBatchRestartRSSleepTime;
+  private float rollingBatchRestartRSRatio;
+  private long restartRsHoldingMetaSleepTime;
+  private float compactTableRatio;
+  private float compactRandomRegionRatio;
+
   @Override
   public ChaosMonkey build() {
 
+    loadProperties();
     // Actions such as compact/flush a table/region,
     // move one region around. They are not so destructive,
     // can be executed more frequently.
     Action[] actions1 = new Action[] {
-        new CompactTableAction(tableName, 0.5f),
-        new CompactRandomRegionOfTableAction(tableName, 0.6f),
+        new CompactTableAction(tableName, compactTableRatio),
+        new CompactRandomRegionOfTableAction(tableName, compactRandomRegionRatio),
         new FlushTableAction(tableName),
         new FlushRandomRegionOfTableAction(tableName),
         new MoveRandomRegionOfTableAction(tableName)
@@ -78,13 +98,15 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
 
     // Destructive actions to mess things around.
     Action[] actions3 = new Action[] {
-        new MoveRegionsOfTableAction(800, tableName),
-        new MoveRandomRegionOfTableAction(800, tableName),
-        new RestartRandomRsAction(60000),
-        new BatchRestartRsAction(5000, 0.5f),
-        new RestartActiveMasterAction(5000),
-        new RollingBatchRestartRsAction(5000, 1.0f),
-        new RestartRsHoldingMetaAction(35000)
+        new MoveRegionsOfTableAction(moveRegionsSleepTime, moveRegionsMaxTime,
+            tableName),
+        new MoveRandomRegionOfTableAction(moveRandomRegionSleepTime, tableName),
+        new RestartRandomRsAction(restartRandomRSSleepTime),
+        new BatchRestartRsAction(batchRestartRSSleepTime, batchRestartRSRatio),
+        new RestartActiveMasterAction(restartActiveMasterSleepTime),
+        new RollingBatchRestartRsAction(rollingBatchRestartRSSleepTime,
+            rollingBatchRestartRSRatio),
+        new RestartRsHoldingMetaAction(restartRsHoldingMetaSleepTime)
     };
 
     // Action to log more info for debugging
@@ -93,11 +115,60 @@ public class SlowDeterministicMonkeyFactory extends MonkeyFactory {
     };
 
     return new PolicyBasedChaosMonkey(util,
-        new PeriodicRandomActionPolicy(60 * 1000, actions1),
-        new PeriodicRandomActionPolicy(90 * 1000, actions2),
+        new PeriodicRandomActionPolicy(action1Period, actions1),
+        new PeriodicRandomActionPolicy(action2Period, actions2),
         new CompositeSequentialPolicy(
-            new DoActionsOncePolicy(150 * 1000, actions3),
-            new PeriodicRandomActionPolicy(150 * 1000, actions3)),
-        new PeriodicRandomActionPolicy(90 * 1000, actions4));
+            new DoActionsOncePolicy(action3Period, actions3),
+            new PeriodicRandomActionPolicy(action3Period, actions3)),
+        new PeriodicRandomActionPolicy(action4Period, actions4));
+  }
+
+  private void loadProperties() {
+
+      action1Period = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.PERIODIC_ACTION1_PERIOD,
+        MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD + ""));
+      action2Period = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.PERIODIC_ACTION2_PERIOD,
+        MonkeyConstants.DEFAULT_PERIODIC_ACTION2_PERIOD + ""));
+      action3Period = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.COMPOSITE_ACTION3_PERIOD,
+        MonkeyConstants.DEFAULT_COMPOSITE_ACTION3_PERIOD + ""));
+      action4Period = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.PERIODIC_ACTION4_PERIOD,
+        MonkeyConstants.DEFAULT_PERIODIC_ACTION4_PERIOD + ""));
+      moveRegionsMaxTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.MOVE_REGIONS_MAX_TIME,
+        MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME + ""));
+      moveRegionsSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.MOVE_REGIONS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_MOVE_REGIONS_SLEEP_TIME + ""));
+      moveRandomRegionSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.MOVE_RANDOM_REGION_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_MOVE_RANDOM_REGION_SLEEP_TIME + ""));
+      restartRandomRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.RESTART_RANDOM_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_RESTART_RANDOM_RS_SLEEP_TIME + ""));
+      batchRestartRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_BATCH_RESTART_RS_SLEEP_TIME + ""));
+      restartActiveMasterSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.RESTART_ACTIVE_MASTER_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_RESTART_ACTIVE_MASTER_SLEEP_TIME + ""));
+      rollingBatchRestartRSSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_SLEEP_TIME + ""));
+      rollingBatchRestartRSRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.ROLLING_BATCH_RESTART_RS_RATIO,
+        MonkeyConstants.DEFAULT_ROLLING_BATCH_RESTART_RS_RATIO + ""));
+      restartRsHoldingMetaSleepTime = Long.parseLong(this.properties.getProperty(
+        MonkeyConstants.RESTART_RS_HOLDING_META_SLEEP_TIME,
+        MonkeyConstants.DEFAULT_RESTART_RS_HOLDING_META_SLEEP_TIME + ""));
+      compactTableRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.COMPACT_TABLE_ACTION_RATIO,
+        MonkeyConstants.DEFAULT_COMPACT_TABLE_ACTION_RATIO + ""));
+      compactRandomRegionRatio = Float.parseFloat(this.properties.getProperty(
+        MonkeyConstants.COMPACT_RANDOM_REGION_RATIO,
+        MonkeyConstants.DEFAULT_COMPACT_RANDOM_REGION_RATIO + ""));
   }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/UnbalanceMonkeyFactory.java
----------------------------------------------------------------------
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 2d2d315..f4ea435 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
@@ -27,15 +27,33 @@ import org.apache.hadoop.hbase.chaos.policies.Policy;
 public class UnbalanceMonkeyFactory extends MonkeyFactory {
   /** How often to introduce the chaos. If too frequent, sequence of kills on minicluster
    * can cause test to fail when Put runs out of retries. */
-  private static final long CHAOS_EVERY_MS = 65 * 1000;
+  private long chaosEveryMilliSec;
+  private long waitForUnbalanceMilliSec;
+  private long waitForKillMilliSec;
+  private long waitAfterBalanceMilliSec;
 
   @Override
   public ChaosMonkey build() {
-    Policy chaosPolicy = new PeriodicRandomActionPolicy(
-        CHAOS_EVERY_MS,
-        new UnbalanceKillAndRebalanceAction()
-    );
+    loadProperties();
+    Policy chaosPolicy = new PeriodicRandomActionPolicy(chaosEveryMilliSec,
+        new UnbalanceKillAndRebalanceAction(waitForUnbalanceMilliSec, waitForKillMilliSec,
+            waitAfterBalanceMilliSec));
 
     return new PolicyBasedChaosMonkey(util, chaosPolicy);
   }
+
+  private void loadProperties() {
+    chaosEveryMilliSec = Long.parseLong(this.properties.getProperty(
+      MonkeyConstants.UNBALANCE_CHAOS_EVERY_MS,
+      MonkeyConstants.DEFAULT_UNBALANCE_CHAOS_EVERY_MS + ""));
+    waitForUnbalanceMilliSec = Long.parseLong(this.properties.getProperty(
+      MonkeyConstants.UNBALANCE_WAIT_FOR_UNBALANCE_MS,
+      MonkeyConstants.DEFAULT_UNBALANCE_WAIT_FOR_UNBALANCE_MS + ""));
+    waitForKillMilliSec = Long.parseLong(this.properties.getProperty(
+      MonkeyConstants.UNBALANCE_WAIT_FOR_KILLS_MS,
+      MonkeyConstants.DEFAULT_UNBALANCE_WAIT_FOR_KILLS_MS + ""));
+    waitAfterBalanceMilliSec = Long.parseLong(this.properties.getProperty(
+      MonkeyConstants.UNBALANCE_WAIT_AFTER_BALANCE_MS,
+      MonkeyConstants.DEFAULT_UNBALANCE_WAIT_AFTER_BALANCE_MS + ""));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
index 7a6b85b..31c67af 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartActiveMasterAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingMetaAction;
 import org.apache.hadoop.hbase.chaos.actions.RestartRsHoldingTableAction;
+import org.apache.hadoop.hbase.chaos.factories.MonkeyConstants;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Put;
@@ -188,7 +189,8 @@ public class IntegrationTestMTTR {
     restartMetaAction = new RestartRsHoldingMetaAction(sleepTime);
 
     // Set up the action that will move the regions of our table.
-    moveRegionAction = new MoveRegionsOfTableAction(sleepTime, tableName.getNameAsString());
+    moveRegionAction = new MoveRegionsOfTableAction(sleepTime,
+        MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME, tableName.getNameAsString());
 
     // Kill the master
     restartMasterAction = new RestartActiveMasterAction(1000);

http://git-wip-us.apache.org/repos/asf/hbase/blob/5764df29/src/main/docbkx/developer.xml
----------------------------------------------------------------------
diff --git a/src/main/docbkx/developer.xml b/src/main/docbkx/developer.xml
index b0491ef..2628a7b 100644
--- a/src/main/docbkx/developer.xml
+++ b/src/main/docbkx/developer.xml
@@ -1011,6 +1011,27 @@ mvn verify</programlisting>
 As you can see from the log, ChaosMonkey started the default PeriodicRandomActionPolicy, which is configured with all the available actions, and ran RestartActiveMaster and RestartRandomRs actions. ChaosMonkey tool, if run from command line, will keep on running until the process is killed.
 </para>
 </section>
+<section xml:id="chaos.monkey.properties">
+    <title>Passing individual Chaos Monkey per-test Settings/Properties</title>
+<para>
+    Since HBase version 1.0.0 (<link xlink:href="https://issues.apache.org/jira/browse/HBASE-11348">HBASE-11348 Make frequency and sleep times of chaos monkeys configurable</link>),
+    the chaos monkeys used to run integration tests can be configured per test run. Users can create a java properties file and
+    and pass this to the chaos monkey with timing configurations. The properties file needs to be in the HBase classpath.
+    The various properties that can be configured and their default values can be found listed in the
+    <classname>org.apache.hadoop.hbase.chaos.factories.MonkeyConstants</classname> class. 
+    If any chaos monkey configuration is missing from the property file, then the default values are assumed.
+    For example:
+    <console><prompt>$</prompt> <userinput>bin/hbase org.apache.hadoop.hbase.IntegrationTestIngest -m slowDeterministic -monkeyProps monkey.properties</userinput></console>
+    The above command will start the integration tests and chaos monkey passing the properties file <filename>monkey.properties</filename>. 
+    Here is an example chaos monkey file:
+    <programlisting>sdm.action1.period=120000
+sdm.action2.period=40000
+move.regions.sleep.time=80000
+move.regions.max.time=1000000
+move.regions.sleep.time=80000
+batch.restart.rs.ratio=0.4f</programlisting>
+</para>
+</section>
 </section>
 </section> <!-- tests -->