You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ec...@apache.org on 2014/10/23 00:19:00 UTC
git commit: HBASE-12314 Add chaos monkey policy to execute two
actions concurrently
Repository: hbase
Updated Branches:
refs/heads/0.98 59d013365 -> 0d1c96589
HBASE-12314 Add chaos monkey policy to execute two actions concurrently
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/0d1c9658
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/0d1c9658
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/0d1c9658
Branch: refs/heads/0.98
Commit: 0d1c96589cf5ea0d3b0799aec73676e5184b4187
Parents: 59d0133
Author: Elliott Clark <ec...@apache.org>
Authored: Tue Oct 21 15:05:59 2014 -0700
Committer: Elliott Clark <ec...@apache.org>
Committed: Wed Oct 22 15:09:54 2014 -0700
----------------------------------------------------------------------
.../hbase/chaos/factories/MonkeyFactory.java | 2 +
.../chaos/factories/NoKillMonkeyFactory.java | 83 +++++++++++++++++
.../policies/TwoConcurrentActionPolicy.java | 97 ++++++++++++++++++++
3 files changed, 182 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d1c9658/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..8f5e610 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
@@ -56,11 +56,13 @@ public abstract class MonkeyFactory {
// TODO: the name has become a misnomer since the default (not-slow) monkey has been removed
public static final String SLOW_DETERMINISTIC = "slowDeterministic";
public static final String UNBALANCE = "unbalance";
+ public static final String NO_KILL = "noKill";
public static Map<String, MonkeyFactory> FACTORIES = ImmutableMap.<String,MonkeyFactory>builder()
.put(CALM, new CalmMonkeyFactory())
.put(SLOW_DETERMINISTIC, new SlowDeterministicMonkeyFactory())
.put(UNBALANCE, new UnbalanceMonkeyFactory())
+ .put(NO_KILL, new NoKillMonkeyFactory())
.build();
public static MonkeyFactory getFactory(String factoryName) {
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d1c9658/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5b4035e
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/factories/NoKillMonkeyFactory.java
@@ -0,0 +1,83 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hbase.chaos.actions.Action;
+import org.apache.hadoop.hbase.chaos.actions.AddColumnAction;
+import org.apache.hadoop.hbase.chaos.actions.ChangeBloomFilterAction;
+import org.apache.hadoop.hbase.chaos.actions.ChangeCompressionAction;
+import org.apache.hadoop.hbase.chaos.actions.ChangeEncodingAction;
+import org.apache.hadoop.hbase.chaos.actions.ChangeVersionsAction;
+import org.apache.hadoop.hbase.chaos.actions.CompactRandomRegionOfTableAction;
+import org.apache.hadoop.hbase.chaos.actions.CompactTableAction;
+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.MergeRandomAdjacentRegionsOfTableAction;
+import org.apache.hadoop.hbase.chaos.actions.MoveRandomRegionOfTableAction;
+import org.apache.hadoop.hbase.chaos.actions.MoveRegionsOfTableAction;
+import org.apache.hadoop.hbase.chaos.actions.RemoveColumnAction;
+import org.apache.hadoop.hbase.chaos.actions.SnapshotTableAction;
+import org.apache.hadoop.hbase.chaos.actions.SplitRandomRegionOfTableAction;
+import org.apache.hadoop.hbase.chaos.monkies.ChaosMonkey;
+import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
+import org.apache.hadoop.hbase.chaos.policies.PeriodicRandomActionPolicy;
+import org.apache.hadoop.hbase.chaos.policies.TwoConcurrentActionPolicy;
+
+/**
+ * Monkey factory to create a ChaosMonkey that will not need access to ssh. It will not
+ * kill any services and it will not perform any restarts.
+ */
+public class NoKillMonkeyFactory extends MonkeyFactory {
+ @Override public ChaosMonkey build() {
+ Action[] actions1 = new Action[] {
+ new CompactTableAction(tableName, 60*1000),
+ new CompactRandomRegionOfTableAction(tableName,0.6f),
+ new FlushTableAction(tableName),
+ new FlushRandomRegionOfTableAction(tableName),
+ new MoveRandomRegionOfTableAction(tableName)
+ };
+
+ Action[] actions2 = new Action[] {
+ new SplitRandomRegionOfTableAction(tableName),
+ new MergeRandomAdjacentRegionsOfTableAction(tableName),
+ new SnapshotTableAction(tableName),
+ new AddColumnAction(tableName),
+ new RemoveColumnAction(tableName, columnFamilies),
+ new ChangeEncodingAction(tableName),
+ new ChangeCompressionAction(tableName),
+ new ChangeBloomFilterAction(tableName),
+ new ChangeVersionsAction(tableName)
+ };
+
+ Action[] actions3 = new Action[] {
+ new MoveRegionsOfTableAction(800,tableName),
+ new MoveRandomRegionOfTableAction(800,tableName),
+ };
+
+ Action[] actions4 = new Action[] {
+ new DumpClusterStatusAction()
+ };
+
+ return new PolicyBasedChaosMonkey(util,
+ new TwoConcurrentActionPolicy(60*1000, actions1, actions2),
+ new PeriodicRandomActionPolicy(90*1000,actions3),
+ new PeriodicRandomActionPolicy(90*1000,actions4));
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/0d1c9658/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/TwoConcurrentActionPolicy.java
----------------------------------------------------------------------
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/TwoConcurrentActionPolicy.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/TwoConcurrentActionPolicy.java
new file mode 100644
index 0000000..5c45d9c
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/policies/TwoConcurrentActionPolicy.java
@@ -0,0 +1,97 @@
+/**
+ * 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.policies;
+
+import org.apache.hadoop.hbase.DaemonThreadFactory;
+import org.apache.hadoop.hbase.chaos.actions.Action;
+import org.apache.hadoop.hbase.chaos.monkies.PolicyBasedChaosMonkey;
+import org.apache.hadoop.util.StringUtils;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+/**
+ * Chaos Monkey policy that will run two different actions at the same time.
+ * A random action from each array of actions will be chosen and then run in parallel.
+ */
+public class TwoConcurrentActionPolicy extends PeriodicPolicy {
+ private final Action[] actionsOne;
+ private final Action[] actionsTwo;
+ private final ExecutorService executor;
+
+ public TwoConcurrentActionPolicy(long sleepTime, Action[] actionsOne, Action[] actionsTwo) {
+ super(sleepTime);
+ this.actionsOne = actionsOne;
+ this.actionsTwo = actionsTwo;
+ executor = Executors.newFixedThreadPool(2,
+ new DaemonThreadFactory("TwoConcurrentAction-"));
+ }
+
+ @Override
+ protected void runOneIteration() {
+ Action actionOne = PolicyBasedChaosMonkey.selectRandomItem(actionsOne);
+ Action actionTwo = PolicyBasedChaosMonkey.selectRandomItem(actionsTwo);
+
+ Future fOne = executor.submit(new ActionRunner(actionOne));
+ Future fTwo = executor.submit(new ActionRunner(actionTwo));
+
+ try {
+ fOne.get();
+ fTwo.get();
+ } catch (InterruptedException e) {
+ LOG.warn("Exception occurred during performing action: "
+ + StringUtils.stringifyException(e));
+ } catch (ExecutionException ex) {
+ LOG.warn("Exception occurred during performing action: "
+ + StringUtils.stringifyException(ex));
+ }
+ }
+
+ @Override
+ public void init(PolicyContext context) throws Exception {
+ super.init(context);
+ for (Action a : actionsOne) {
+ a.init(context);
+ }
+ for (Action a : actionsTwo) {
+ a.init(context);
+ }
+ }
+
+ private static class ActionRunner implements Runnable {
+
+ private final Action action;
+
+ public ActionRunner(Action action) {
+
+ this.action = action;
+ }
+
+ @Override public void run() {
+ try {
+ action.perform();
+ } catch (Exception ex) {
+ LOG.warn("Exception occurred during performing action: "
+ + StringUtils.stringifyException(ex));
+ }
+ }
+ }
+}