View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Chaos monkey that given multiple policies will run actions against the cluster.
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     * Construct a new ChaosMonkey
56     * @param util the HBaseIntegrationTestingUtility already configured
57     * @param policies custom policies to use
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    /** Selects a random item from the given items */
85    public static <T> T selectRandomItem(T[] items) {
86      return items[RandomUtils.nextInt(items.length)];
87    }
88  
89    /** Selects a random item from the given items with weights*/
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     //warn: O(n)
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   /** Selects and returns ceil(ratio * items.length) random items from the given array */
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     // stop accepting new work (shouldn't be any with a fixed-size pool)
135     monkeyThreadPool.shutdown();
136     // notify all executing policies that it's time to halt.
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     // TODO: we can look at the actions, and decide to do the restore cluster or not based on them.
155     return true;
156   }
157 }