HBASE-12314 Add chaos monkey policy to execute two actions concurrently

This commit is contained in:
Elliott Clark 2014-10-21 15:05:59 -07:00
parent c623b04122
commit 84f3549d5e
4 changed files with 186 additions and 1 deletions

View File

@ -68,6 +68,7 @@ public abstract class MonkeyFactory {
public static final String UNBALANCE = "unbalance";
public static final String SERVER_KILLING = "serverKilling";
public static final String STRESS_AM = "stressAM";
public static final String NO_KILL = "noKill";
public static Map<String, MonkeyFactory> FACTORIES = ImmutableMap.<String,MonkeyFactory>builder()
.put(CALM, new CalmMonkeyFactory())
@ -75,6 +76,7 @@ public abstract class MonkeyFactory {
.put(UNBALANCE, new UnbalanceMonkeyFactory())
.put(SERVER_KILLING, new ServerKillingMonkeyFactory())
.put(STRESS_AM, new StressAssignmentManagerMonkeyFactory())
.put(NO_KILL, new NoKillMonkeyFactory())
.build();
public static MonkeyFactory getFactory(String factoryName) {

View File

@ -0,0 +1,87 @@
/**
* 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, MonkeyConstants.DEFAULT_PERIODIC_ACTION1_PERIOD),
new CompactRandomRegionOfTableAction(tableName,
MonkeyConstants.DEFAULT_COMPACT_RANDOM_REGION_RATIO),
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(MonkeyConstants.DEFAULT_MOVE_REGIONS_SLEEP_TIME,
MonkeyConstants.DEFAULT_MOVE_REGIONS_MAX_TIME,
tableName),
new MoveRandomRegionOfTableAction(MonkeyConstants.DEFAULT_RESTART_ACTIVE_MASTER_SLEEP_TIME,
tableName),
};
Action[] actions4 = new Action[] {
new DumpClusterStatusAction()
};
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));
}
}

View File

@ -45,7 +45,6 @@ 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 {

View File

@ -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));
}
}
}
}