1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.chaos.monkies;
20
21 import com.google.common.util.concurrent.ThreadFactoryBuilder;
22 import java.util.Arrays;
23 import java.util.Collection;
24 import java.util.Collections;
25 import java.util.List;
26
27 import java.util.Objects;
28 import java.util.concurrent.ExecutorService;
29 import java.util.concurrent.Executors;
30 import java.util.concurrent.TimeUnit;
31 import org.apache.commons.lang.math.RandomUtils;
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.IntegrationTestingUtility;
35 import org.apache.hadoop.hbase.chaos.policies.Policy;
36 import org.apache.hadoop.hbase.util.Pair;
37
38
39
40
41 public class PolicyBasedChaosMonkey extends ChaosMonkey {
42
43 private static final Log LOG = LogFactory.getLog(PolicyBasedChaosMonkey.class);
44 private static final long ONE_SEC = 1000;
45 private static final long ONE_MIN = 60 * ONE_SEC;
46
47 public static final long TIMEOUT = ONE_MIN;
48
49 final IntegrationTestingUtility util;
50
51 private final Policy[] policies;
52 private final ExecutorService monkeyThreadPool;
53
54
55
56
57
58
59 public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Collection<Policy> policies) {
60 this(util, policies.toArray(new Policy[0]));
61 }
62
63 public PolicyBasedChaosMonkey(IntegrationTestingUtility util, Policy... policies) {
64 this.util = Objects.requireNonNull(util);
65 this.policies = Objects.requireNonNull(policies);
66 if (policies.length == 0) {
67 throw new IllegalArgumentException("policies may not be empty");
68 }
69 this.monkeyThreadPool = buildMonkeyThreadPool(policies.length);
70 }
71
72 private static ExecutorService buildMonkeyThreadPool(final int size) {
73 return Executors.newFixedThreadPool(size, new ThreadFactoryBuilder()
74 .setDaemon(false)
75 .setNameFormat("ChaosMonkey-%d")
76 .setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
77 @Override public void uncaughtException(Thread t, Throwable e) {
78 throw new RuntimeException(e);
79 }
80 })
81 .build());
82 }
83
84
85 public static <T> T selectRandomItem(T[] items) {
86 return items[RandomUtils.nextInt(items.length)];
87 }
88
89
90 public static <T> T selectWeightedRandomItem(List<Pair<T, Integer>> items) {
91 int totalWeight = 0;
92 for (Pair<T, Integer> pair : items) {
93 totalWeight += pair.getSecond();
94 }
95
96 int cutoff = RandomUtils.nextInt(totalWeight);
97 int cummulative = 0;
98 T item = null;
99
100
101 for (int i=0; i<items.size(); i++) {
102 int curWeight = items.get(i).getSecond();
103 if ( cutoff < cummulative + curWeight) {
104 item = items.get(i).getFirst();
105 break;
106 }
107 cummulative += curWeight;
108 }
109
110 return item;
111 }
112
113
114 public static <T> List<T> selectRandomItems(T[] items, float ratio) {
115 int selectedNumber = (int)Math.ceil(items.length * ratio);
116
117 List<T> originalItems = Arrays.asList(items);
118 Collections.shuffle(originalItems);
119 int startIndex = RandomUtils.nextInt(items.length - selectedNumber);
120 return originalItems.subList(startIndex, startIndex + selectedNumber);
121 }
122
123 @Override
124 public void start() throws Exception {
125 final Policy.PolicyContext context = new Policy.PolicyContext(this.util);
126 for (final Policy policy : policies) {
127 policy.init(context);
128 monkeyThreadPool.execute(policy);
129 }
130 }
131
132 @Override
133 public void stop(String why) {
134
135 monkeyThreadPool.shutdown();
136
137 for (Policy policy : policies) {
138 policy.stop(why);
139 }
140 }
141
142 @Override
143 public boolean isStopped() {
144 return monkeyThreadPool.isTerminated();
145 }
146
147 @Override
148 public void waitForStop() throws InterruptedException {
149 monkeyThreadPool.awaitTermination(1, TimeUnit.MINUTES);
150 }
151
152 @Override
153 public boolean isDestructive() {
154
155 return true;
156 }
157 }