From f80c4c24b6aefb4c58bb0129298fc89ba16f7e1a Mon Sep 17 00:00:00 2001 From: Lokesh Khurana Date: Thu, 16 Apr 2020 07:43:20 +0530 Subject: [PATCH] HBASE-24193 : BackPort HBASE-18651 to branch-1 (#1521) Signed-off-by: Reid Chan --- .../hbase/chaos/util/ChaosMonkeyRunner.java | 21 +++- .../hadoop/hbase/chaos/util/Monkeys.java | 98 +++++++++++++++++++ .../hbase/test/IntegrationTestMonkeys.java | 67 +++++++++++++ 3 files changed, 184 insertions(+), 2 deletions(-) create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java create mode 100644 hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java index 1ee7d936c51..d559e88d6b3 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.java @@ -92,8 +92,25 @@ public class ChaosMonkeyRunner extends AbstractHBaseTool { protected int doWork() throws Exception { setUpCluster(); getAndStartMonkey(); - while (true) {// loop here until got killed - Thread.sleep(10000); + while (!monkey.isStopped()) { + // loop here until got killed + try{ + Thread.sleep(5000); + } 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() { + if (monkey != null) { + monkey.stop("Program Control"); } } diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java new file mode 100644 index 00000000000..d3366f8e5b5 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java @@ -0,0 +1,98 @@ +/** + * 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 com.google.common.base.Preconditions; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +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.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.IntegrationTestingUtility; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class can be used to control chaos monkeys life cycle. + */ +public class Monkeys implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(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 = new Runnable() { + @Override public void run() { + try { + monkeyRunner.getAndStartMonkey(); + } catch (Exception e) { + LOG.error("Exception occurred 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 occurred while stopping chaos monkeys " + e); + } + } +} diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java new file mode 100644 index 00000000000..7b217f6d975 --- /dev/null +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java @@ -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); + } +}