You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2017/09/24 00:34:41 UTC

[41/47] hbase git commit: HBASE-18651 Let ChaosMonkeyRunner expose the chaos monkey runner it creates

HBASE-18651 Let ChaosMonkeyRunner expose the chaos monkey runner it creates

Signed-off-by: Mike Drob <md...@apache.org>
Signed-off-by: Ted Yu <te...@apache.org>


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

Branch: refs/heads/HBASE-18467
Commit: 5f238b3ef46e7d5f31a9fb2dfff0111085af835d
Parents: e393599
Author: Reid Chan <re...@outlook.com>
Authored: Thu Sep 21 11:30:14 2017 +0800
Committer: Mike Drob <md...@apache.org>
Committed: Thu Sep 21 21:19:25 2017 -0500

----------------------------------------------------------------------
 .../hbase/chaos/util/ChaosMonkeyRunner.java     | 19 +++-
 .../apache/hadoop/hbase/chaos/util/Monkeys.java | 96 ++++++++++++++++++++
 .../hbase/test/IntegrationTestMonkeys.java      | 67 ++++++++++++++
 3 files changed, 178 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/ChaosMonkeyRunner.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 5911085..d72111f 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
@@ -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 {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java
----------------------------------------------------------------------
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 0000000..1ce4356
--- /dev/null
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/util/Monkeys.java
@@ -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);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hbase/blob/5f238b3e/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestMonkeys.java
----------------------------------------------------------------------
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 0000000..7b217f6
--- /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);
+  }
+}