HBASE-18651 Let ChaosMonkeyRunner expose the chaos monkey runner it creates
Signed-off-by: Mike Drob <mdrob@apache.org> Signed-off-by: Ted Yu <tedyu@apache.org>
This commit is contained in:
parent
c1f5122fab
commit
3a02a2a80d
|
@ -52,7 +52,6 @@ public class ChaosMonkeyRunner extends AbstractHBaseTool {
|
|||
protected boolean noClusterCleanUp = false;
|
||||
private String tableName = "ChaosMonkeyRunner.tableName";
|
||||
private String familyName = "ChaosMonkeyRunner.familyName";
|
||||
private volatile boolean stop = false;
|
||||
|
||||
@Override
|
||||
public void addOptions() {
|
||||
|
@ -93,14 +92,26 @@ public class ChaosMonkeyRunner extends AbstractHBaseTool {
|
|||
protected int doWork() throws Exception {
|
||||
setUpCluster();
|
||||
getAndStartMonkey();
|
||||
while (!stop) {// loop here until got killed
|
||||
Thread.sleep(10000);
|
||||
while (!monkey.isStopped()) {
|
||||
// loop here until got killed
|
||||
try {
|
||||
// TODO: make sleep time configurable
|
||||
Thread.sleep(5000); // 5 seconds
|
||||
} catch (InterruptedException ite) {
|
||||
// Chaos monkeys got interrupted.
|
||||
// It is ok to stop monkeys and exit.
|
||||
monkey.stop("Interruption occurred.");
|
||||
break;
|
||||
}
|
||||
}
|
||||
monkey.waitForStop();
|
||||
return 0;
|
||||
}
|
||||
|
||||
public void stopRunner() {
|
||||
stop = true;
|
||||
if (monkey != null) {
|
||||
monkey.stop("Program Control");
|
||||
}
|
||||
}
|
||||
|
||||
public void setUpCluster() throws Exception {
|
||||
|
|
|
@ -0,0 +1,96 @@
|
|||
/**
|
||||
* 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.util;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.IntegrationTestingUtility;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hbase.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder;
|
||||
|
||||
/**
|
||||
* This class can be used to control chaos monkeys life cycle.
|
||||
*/
|
||||
public class Monkeys implements Closeable {
|
||||
private static final Log LOG = LogFactory.getLog(Monkeys.class);
|
||||
|
||||
private final Configuration conf;
|
||||
private final ChaosMonkeyRunner monkeyRunner;
|
||||
private final Runnable runner;
|
||||
private final ExecutorService executor;
|
||||
|
||||
public Monkeys() {
|
||||
this(HBaseConfiguration.create());
|
||||
}
|
||||
|
||||
public Monkeys(Configuration conf) {
|
||||
this.conf = Preconditions.checkNotNull(conf, "Should specify a configuration");
|
||||
this.monkeyRunner = new ChaosMonkeyRunner();
|
||||
this.runner = () -> {
|
||||
try {
|
||||
monkeyRunner.getAndStartMonkey();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Exception occured when running chaos monkeys: ", e);
|
||||
}
|
||||
};
|
||||
this.executor = Executors.newSingleThreadExecutor(new ThreadFactoryBuilder()
|
||||
.setDaemon(true).setNameFormat("ChaosMonkey").build());
|
||||
IntegrationTestingUtility.setUseDistributedCluster(this.conf);
|
||||
}
|
||||
|
||||
public void addResource(Configuration otherConf) {
|
||||
conf.addResource(otherConf);
|
||||
monkeyRunner.setConf(conf);
|
||||
}
|
||||
|
||||
public void addResource(String otherConf) {
|
||||
conf.addResource(otherConf);
|
||||
monkeyRunner.setConf(conf);
|
||||
}
|
||||
|
||||
public void startChaos() {
|
||||
executor.execute(runner);
|
||||
LOG.info("Chaos monkeys are running.");
|
||||
}
|
||||
|
||||
public void stopChaos() {
|
||||
monkeyRunner.stopRunner();
|
||||
LOG.info("Chaos monkeys are stopped.");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
executor.shutdown();
|
||||
try {
|
||||
// wait 10 seconds.
|
||||
executor.awaitTermination(10, TimeUnit.SECONDS);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
LOG.warn("Interruption occured while stopping chaos monkeys " + e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* 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.test;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.IntegrationTestingUtility;
|
||||
import org.apache.hadoop.hbase.chaos.util.ChaosMonkeyRunner;
|
||||
import org.apache.hadoop.hbase.chaos.util.Monkeys;
|
||||
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
|
||||
import org.apache.hadoop.util.ToolRunner;
|
||||
import org.junit.Test;
|
||||
import org.junit.experimental.categories.Category;
|
||||
|
||||
/**
|
||||
* This is an integration test for showing a simple usage of how to use {@link Monkeys}
|
||||
* to control {@link ChaosMonkeyRunner}.
|
||||
*/
|
||||
@Category(IntegrationTests.class)
|
||||
public class IntegrationTestMonkeys extends ChaosMonkeyRunner {
|
||||
private static final int RUN_SECS = 15 * 1000;
|
||||
private static final int WAIT_SECS = 10 * 1000;
|
||||
|
||||
@Override
|
||||
protected int doWork() throws Exception {
|
||||
super.setUpCluster();
|
||||
runMonkeys();
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void runMonkeys() throws Exception {
|
||||
try (Monkeys monkeys = new Monkeys()) {
|
||||
for (int i = 0; i < 2; i++) {
|
||||
monkeys.startChaos();
|
||||
Thread.sleep(RUN_SECS);
|
||||
monkeys.stopChaos();
|
||||
Thread.sleep(WAIT_SECS);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
// Run chaos monkeys 15 seconds, then stop them.
|
||||
// After 10 seconds, run chaos monkeys again.
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
IntegrationTestingUtility.setUseDistributedCluster(conf);
|
||||
int exitCode = ToolRunner.run(conf, new IntegrationTestMonkeys(), args);
|
||||
System.exit(exitCode);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue