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 <bharathv@apache.org> Signed-off-by: Viraj Jasani <vjasani@apache.org>
This commit is contained in:
parent
620470607e
commit
89cf76c2cd
|
@ -1,4 +1,4 @@
|
||||||
/**
|
/*
|
||||||
* Licensed to the Apache Software Foundation (ASF) under one
|
* Licensed to the Apache Software Foundation (ASF) under one
|
||||||
* or more contributor license agreements. See the NOTICE file
|
* or more contributor license agreements. See the NOTICE file
|
||||||
* distributed with this work for additional information
|
* distributed with this work for additional information
|
||||||
|
@ -22,14 +22,18 @@ import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.Properties;
|
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.commons.lang3.RandomUtils;
|
||||||
import org.apache.hadoop.hbase.IntegrationTestingUtility;
|
import org.apache.hadoop.hbase.IntegrationTestingUtility;
|
||||||
import org.apache.hadoop.hbase.chaos.policies.Policy;
|
import org.apache.hadoop.hbase.chaos.policies.Policy;
|
||||||
import org.apache.hadoop.hbase.util.Pair;
|
import org.apache.hadoop.hbase.util.Pair;
|
||||||
import org.slf4j.Logger;
|
import org.slf4j.Logger;
|
||||||
import org.slf4j.LoggerFactory;
|
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.
|
* 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 Logger LOG = LoggerFactory.getLogger(PolicyBasedChaosMonkey.class);
|
||||||
private static final long ONE_SEC = 1000;
|
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;
|
private static final long ONE_MIN = 60 * ONE_SEC;
|
||||||
|
|
||||||
public static final long TIMEOUT = ONE_MIN;
|
public static final long TIMEOUT = ONE_MIN;
|
||||||
|
@ -46,6 +49,9 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
|
||||||
final IntegrationTestingUtility util;
|
final IntegrationTestingUtility util;
|
||||||
final Properties monkeyProps;
|
final Properties monkeyProps;
|
||||||
|
|
||||||
|
private final Policy[] policies;
|
||||||
|
private final ExecutorService monkeyThreadPool;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Construct a new ChaosMonkey
|
* Construct a new ChaosMonkey
|
||||||
* @param util the HBaseIntegrationTestingUtility already configured
|
* @param util the HBaseIntegrationTestingUtility already configured
|
||||||
|
@ -60,19 +66,30 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
|
||||||
}
|
}
|
||||||
|
|
||||||
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
|
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
|
||||||
Policy... policies) {
|
Collection<Policy> policies) {
|
||||||
this.monkeyProps = monkeyProps;
|
this(monkeyProps, util, policies.toArray(new Policy[0]));
|
||||||
this.util = util;
|
|
||||||
this.policies = policies;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
|
public PolicyBasedChaosMonkey(Properties monkeyProps, IntegrationTestingUtility util,
|
||||||
Collection<Policy> policies) {
|
Policy... policies) {
|
||||||
this.monkeyProps = monkeyProps;
|
this.monkeyProps = monkeyProps;
|
||||||
this.util = util;
|
this.util = Objects.requireNonNull(util);
|
||||||
this.policies = policies.toArray(new Policy[policies.size()]);
|
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 */
|
/** Selects a random item from the given items */
|
||||||
public static <T> T selectRandomItem(T[] items) {
|
public static <T> T selectRandomItem(T[] items) {
|
||||||
|
@ -114,27 +131,20 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
|
||||||
return originalItems.subList(startIndex, startIndex + selectedNumber);
|
return originalItems.subList(startIndex, startIndex + selectedNumber);
|
||||||
}
|
}
|
||||||
|
|
||||||
private Policy[] policies;
|
|
||||||
private Thread[] monkeyThreads;
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void start() throws Exception {
|
public void start() throws Exception {
|
||||||
monkeyThreads = new Thread[policies.length];
|
final Policy.PolicyContext context = new Policy.PolicyContext(monkeyProps, util);
|
||||||
Policy.PolicyContext context = new Policy.PolicyContext(monkeyProps, this.util);
|
for (final Policy policy : policies) {
|
||||||
for (int i=0; i<policies.length; i++) {
|
policy.init(context);
|
||||||
policies[i].init(context);
|
monkeyThreadPool.execute(policy);
|
||||||
Thread monkeyThread = new Thread(policies[i], "ChaosMonkey");
|
|
||||||
monkeyThread.start();
|
|
||||||
monkeyThreads[i] = monkeyThread;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void stop(String why) {
|
public void stop(String why) {
|
||||||
if (policies == null) {
|
// stop accepting new work (shouldn't be any with a fixed-size pool)
|
||||||
return;
|
monkeyThreadPool.shutdown();
|
||||||
}
|
// notify all executing policies that it's time to halt.
|
||||||
|
|
||||||
for (Policy policy : policies) {
|
for (Policy policy : policies) {
|
||||||
policy.stop(why);
|
policy.stop(why);
|
||||||
}
|
}
|
||||||
|
@ -142,22 +152,12 @@ public class PolicyBasedChaosMonkey extends ChaosMonkey {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isStopped() {
|
public boolean isStopped() {
|
||||||
return policies[0].isStopped();
|
return monkeyThreadPool.isTerminated();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Wait for ChaosMonkey to stop.
|
|
||||||
* @throws InterruptedException
|
|
||||||
*/
|
|
||||||
@Override
|
@Override
|
||||||
public void waitForStop() throws InterruptedException {
|
public void waitForStop() throws InterruptedException {
|
||||||
if (monkeyThreads == null) {
|
monkeyThreadPool.awaitTermination(1, TimeUnit.MINUTES);
|
||||||
return;
|
|
||||||
}
|
|
||||||
for (Thread monkeyThread : monkeyThreads) {
|
|
||||||
// TODO: bound the wait time per policy
|
|
||||||
monkeyThread.join();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
Loading…
Reference in New Issue