You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/07/21 00:00:18 UTC
[hbase] branch branch-2 updated: HBASE-24658 Update
PolicyBasedChaosMonkey to handle uncaught exceptions
This is an automated email from the ASF dual-hosted git repository.
ndimiduk pushed a commit to branch branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/branch-2 by this push:
new 89cf76c HBASE-24658 Update PolicyBasedChaosMonkey to handle uncaught exceptions
89cf76c is described below
commit 89cf76c2cd7ada76af1794401ddcf597d591f6f1
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Mon Jun 29 16:35:26 2020 -0700
HBASE-24658 Update PolicyBasedChaosMonkey to handle uncaught exceptions
Running `ServerKillingChaosMonkey` via `RESTApiClusterManager` for any
duration of time slowly leaks region servers. I see failures on the
RESTApi side go unreported on the ChaosMonkey side. It seems like
`RuntimeException`s are being thrown and lost.
`PolicyBasedChaosMonkey` uses a primitive means of thread management
anyway. Update to use a thread pool, thread groups, and an
uncaughtExceptionHandler.
Signed-off-by: Bharath Vissapragada <bh...@apache.org>
Signed-off-by: Viraj Jasani <vj...@apache.org>
---
.../chaos/monkies/PolicyBasedChaosMonkey.java | 72 +++++++++++-----------
1 file changed, 36 insertions(+), 36 deletions(-)
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
index dc2ac13..e3e2e49 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/chaos/monkies/PolicyBasedChaosMonkey.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -22,14 +22,18 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
+import java.util.Objects;
import java.util.Properties;
-
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.chaos.policies.Policy;
import org.apache.hadoop.hbase.util.Pair;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
/**
* Chaos monkey that given multiple policies will run actions against the cluster.
@@ -38,7 +42,6 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
private static final Logger LOG = LoggerFactory.getLogger(PolicyBasedChaosMonkey.class);
private static final long ONE_SEC = 1000;
- private static final long FIVE_SEC = 5 * ONE_SEC;
private static final long ONE_MIN = 60 * ONE_SEC;
public static final long TIMEOUT = ONE_MIN;
@@ -46,6 +49,9 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
final IntegrationTestingUtility util;
final Properties monkeyProps;
+ private final Policy[] policies;
+ private final ExecutorService monkeyThreadPool;
+
/**
* Construct a new ChaosMonkey
* @param util the HBaseIntegrationTestingUtility already configured
@@ -60,19 +66,30 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
}
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
- Policy... policies) {
- this.monkeyProps = monkeyProps;
- this.util = util;
- this.policies = policies;
+ Collection<Policy> policies) {
+ this(monkeyProps, util, policies.toArray(new Policy[0]));
}
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
- Collection<Policy> policies) {
+ Policy... policies) {
this.monkeyProps = monkeyProps;
- this.util = util;
- this.policies = policies.toArray(new Policy[policies.size()]);
+ this.util = Objects.requireNonNull(util);
+ this.policies = Objects.requireNonNull(policies);
+ if (policies.length == 0) {
+ throw new IllegalArgumentException("policies may not be empty");
+ }
+ this.monkeyThreadPool = buildMonkeyThreadPool(policies.length);
}
+ private static ExecutorService buildMonkeyThreadPool(final int size) {
+ return Executors.newFixedThreadPool(size, new ThreadFactoryBuilder()
+ .setDaemon(false)
+ .setNameFormat("ChaosMonkey-%d")
+ .setUncaughtExceptionHandler((t, e) -> {
+ throw new RuntimeException(e);
+ })
+ .build());
+ }
/** Selects a random item from the given items */
public static <T> T selectRandomItem(T[] items) {
@@ -114,27 +131,20 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
return originalItems.subList(startIndex, startIndex + selectedNumber);
}
- private Policy[] policies;
- private Thread[] monkeyThreads;
-
@Override
public void start() throws Exception {
- monkeyThreads = new Thread[policies.length];
- Policy.PolicyContext context = new Policy.PolicyContext(monkeyProps, this.util);
- for (int i=0; i<policies.length; i++) {
- policies[i].init(context);
- Thread monkeyThread = new Thread(policies[i], "ChaosMonkey");
- monkeyThread.start();
- monkeyThreads[i] = monkeyThread;
+ final Policy.PolicyContext context = new Policy.PolicyContext(monkeyProps, util);
+ for (final Policy policy : policies) {
+ policy.init(context);
+ monkeyThreadPool.execute(policy);
}
}
@Override
public void stop(String why) {
- if (policies == null) {
- return;
- }
-
+ // stop accepting new work (shouldn't be any with a fixed-size pool)
+ monkeyThreadPool.shutdown();
+ // notify all executing policies that it's time to halt.
for (Policy policy : policies) {
policy.stop(why);
}
@@ -142,22 +152,12 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
@Override
public boolean isStopped() {
- return policies[0].isStopped();
+ return monkeyThreadPool.isTerminated();
}
- /**
- * Wait for ChaosMonkey to stop.
- * @throws InterruptedException
- */
@Override
public void waitForStop() throws InterruptedException {
- if (monkeyThreads == null) {
- return;
- }
- for (Thread monkeyThread : monkeyThreads) {
- // TODO: bound the wait time per policy
- monkeyThread.join();
- }
+ monkeyThreadPool.awaitTermination(1, TimeUnit.MINUTES);
}
@Override