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.util;
20  
21  import java.io.IOException;
22  import java.util.ArrayList;
23  import java.util.Arrays;
24  import java.util.Collection;
25  import java.util.LinkedList;
26  import java.util.List;
27  import java.util.Map;
28  import java.util.Queue;
29  import java.util.Random;
30  
31  import org.apache.commons.cli.CommandLine;
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.hbase.ClusterStatus;
36  import org.apache.hadoop.hbase.HBaseCluster;
37  import org.apache.hadoop.hbase.HBaseConfiguration;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.HServerLoad;
40  import org.apache.hadoop.hbase.IntegrationTestDataIngestWithChaosMonkey;
41  import org.apache.hadoop.hbase.IntegrationTestingUtility;
42  import org.apache.hadoop.hbase.ServerName;
43  import org.apache.hadoop.hbase.Stoppable;
44  import org.apache.hadoop.hbase.client.HBaseAdmin;
45  import org.apache.hadoop.hbase.client.HTable;
46  import org.apache.hadoop.util.StringUtils;
47  import org.apache.hadoop.util.ToolRunner;
48  
49  import com.google.common.collect.Lists;
50  import com.google.common.collect.Maps;
51  
52  /**
53   * A utility to injects faults in a running cluster.
54   * <p>
55   * ChaosMonkey defines Action's and Policy's. Actions are sequences of events, like
56   *  - Select a random server to kill
57   *  - Sleep for 5 sec
58   *  - Start the server on the same host
59   * Actions can also be complex events, like rolling restart of all of the servers.
60   * <p>
61   * Policies on the other hand are responsible for executing the actions based on a strategy.
62   * The default policy is to execute a random action every minute based on predefined action
63   * weights. ChaosMonkey executes predefined named policies until it is stopped. More than one
64   * policy can be active at any time.
65   * <p>
66   * Chaos monkey can be run from the command line, or can be invoked from integration tests.
67   * See {@link IntegrationTestDataIngestWithChaosMonkey} or other integration tests that use
68   * chaos monkey for code examples.
69   * <p>
70   * ChaosMonkey class is indeed inspired by the Netflix's same-named tool:
71   * http://techblog.netflix.com/2012/07/chaos-monkey-released-into-wild.html
72   */
73  public class ChaosMonkey extends AbstractHBaseTool implements Stoppable {
74  
75    private static final Log LOG = LogFactory.getLog(ChaosMonkey.class);
76  
77    private static final long ONE_SEC = 1000;
78    private static final long FIVE_SEC = 5 * ONE_SEC;
79    private static final long ONE_MIN = 60 * ONE_SEC;
80    private static final long TIMEOUT = ONE_MIN;
81  
82    final IntegrationTestingUtility util;
83  
84    /**
85     * Construct a new ChaosMonkey
86     * @param util the HBaseIntegrationTestingUtility already configured
87     */
88    public ChaosMonkey(IntegrationTestingUtility util) {
89  	  this.util = util;
90    }
91    
92    /**
93     * Construct a new ChaosMonkey
94     * @param util the HBaseIntegrationTestingUtility already configured
95     * @param policies names of pre-defined policies to use
96     */
97    public ChaosMonkey(IntegrationTestingUtility util, String... policies) {
98      this.util = util;
99      setPoliciesByName(policies);
100   }
101 
102   /**
103    * Construct a new ChaosMonkey
104    * @param util the HBaseIntegrationTestingUtility already configured
105    * @param policies custom policies to use
106    */
107   public ChaosMonkey(IntegrationTestingUtility util, Policy... policies) {
108     this.util = util;
109     this.policies = policies;
110   }
111 
112   private void setPoliciesByName(String... policies) {
113     this.policies = new Policy[policies.length];
114     for (int i=0; i < policies.length; i++) {
115       this.policies[i] = NAMED_POLICIES.get(policies[i]);
116     }
117   }
118 
119   private void setPolicies(Policy... policies) {
120     this.policies = new Policy[policies.length];
121     for (int i = 0; i < policies.length; i++) {
122       this.policies[i] = policies[i];
123     }
124   }
125 
126   /**
127    * Context for Action's
128    */
129   public static class ActionContext {
130     private IntegrationTestingUtility util;
131 
132     public ActionContext(IntegrationTestingUtility util) {
133       this.util = util;
134     }
135 
136     public IntegrationTestingUtility getHBaseIntegrationTestingUtility() {
137       return util;
138     }
139 
140     public HBaseCluster getHBaseCluster() {
141       return util.getHBaseClusterInterface();
142     }
143   }
144 
145   /**
146    * A (possibly mischievous) action that the ChaosMonkey can perform.
147    */
148   public static class Action {
149     // TODO: interesting question - should actions be implemented inside
150     //       ChaosMonkey, or outside? If they are inside (initial), the class becomes
151     //       huge and all-encompassing; if they are outside ChaosMonkey becomes just
152     //       a random task scheduler. For now, keep inside.
153 
154     protected ActionContext context;
155     protected HBaseCluster cluster;
156     protected ClusterStatus initialStatus;
157     protected ServerName[] initialServers;
158 
159     public void init(ActionContext context) throws IOException {
160       this.context = context;
161       cluster = context.getHBaseCluster();
162       initialStatus = cluster.getInitialClusterStatus();
163       Collection<ServerName> regionServers = initialStatus.getServers();
164       initialServers = regionServers.toArray(new ServerName[regionServers.size()]);
165     }
166 
167     public void perform() throws Exception { };
168 
169     // TODO: perhaps these methods should be elsewhere?
170     /** Returns current region servers */
171     protected ServerName[] getCurrentServers() throws IOException {
172       Collection<ServerName> regionServers = cluster.getClusterStatus().getServers();
173       return regionServers.toArray(new ServerName[regionServers.size()]);
174     }
175 
176     protected void killMaster(ServerName server) throws IOException {
177       LOG.info("Killing master:" + server);
178       cluster.killMaster(server);
179       cluster.waitForMasterToStop(server, TIMEOUT);
180       LOG.info("Killed master server:" + server);
181     }
182 
183     protected void startMaster(ServerName server) throws IOException {
184       LOG.info("Starting master:" + server.getHostname());
185       cluster.startMaster(server.getHostname());
186       cluster.waitForActiveAndReadyMaster(TIMEOUT);
187       LOG.info("Started master: " + server);
188     }
189 
190     protected void killRs(ServerName server) throws IOException {
191       LOG.info("Killing region server:" + server);
192       cluster.killRegionServer(server);
193       cluster.waitForRegionServerToStop(server, TIMEOUT);
194       LOG.info("Killed region server:" + server + ". Reported num of rs:"
195           + cluster.getClusterStatus().getServersSize());
196     }
197 
198     protected void startRs(ServerName server) throws IOException {
199       LOG.info("Starting region server:" + server.getHostname());
200       cluster.startRegionServer(server.getHostname());
201       cluster.waitForRegionServerToStart(server.getHostname(), TIMEOUT);
202       LOG.info("Started region server:" + server + ". Reported num of rs:"
203           + cluster.getClusterStatus().getServersSize());
204     }
205   }
206 
207   private static class RestartActionBase extends Action {
208     long sleepTime; // how long should we sleep
209 
210     public RestartActionBase(long sleepTime) {
211       this.sleepTime = sleepTime;
212     }
213 
214     void sleep(long sleepTime) {
215       LOG.info("Sleeping for:" + sleepTime);
216       Threads.sleep(sleepTime);
217     }
218 
219     void restartMaster(ServerName server, long sleepTime) throws IOException {
220       sleepTime = Math.max(sleepTime, 1000);
221       killMaster(server);
222       sleep(sleepTime);
223       startMaster(server);
224     }
225 
226     void restartRs(ServerName server, long sleepTime) throws IOException {
227       sleepTime = Math.max(sleepTime, 1000);
228       killRs(server);
229       sleep(sleepTime);
230       startRs(server);
231     }
232   }
233 
234   public static class RestartActiveMaster extends RestartActionBase {
235     public RestartActiveMaster(long sleepTime) {
236       super(sleepTime);
237     }
238     @Override
239     public void perform() throws Exception {
240       LOG.info("Performing action: Restart active master");
241 
242       ServerName master = cluster.getClusterStatus().getMaster();
243       restartMaster(master, sleepTime);
244     }
245   }
246 
247   public static class RestartRandomRs extends RestartActionBase {
248     public RestartRandomRs(long sleepTime) {
249       super(sleepTime);
250     }
251 
252     @Override
253     public void perform() throws Exception {
254       LOG.info("Performing action: Restart random region server");
255       ServerName server = selectRandomItem(getCurrentServers());
256 
257       restartRs(server, sleepTime);
258     }
259   }
260 
261   public static class RestartRsHoldingMeta extends RestartActionBase {
262     public RestartRsHoldingMeta(long sleepTime) {
263       super(sleepTime);
264     }
265     @Override
266     public void perform() throws Exception {
267       LOG.info("Performing action: Restart region server holding META");
268       ServerName server = cluster.getServerHoldingMeta();
269       if (server == null) {
270         LOG.warn("No server is holding .META. right now.");
271         return;
272       }
273       restartRs(server, sleepTime);
274     }
275   }
276 
277   public static class RestartRsHoldingRoot extends RestartRandomRs {
278     public RestartRsHoldingRoot(long sleepTime) {
279       super(sleepTime);
280     }
281     @Override
282     public void perform() throws Exception {
283       LOG.info("Performing action: Restart region server holding ROOT");
284       ServerName server = cluster.getServerHoldingRoot();
285       if (server == null) {
286         LOG.warn("No server is holding -ROOT- right now.");
287         return;
288       }
289       restartRs(server, sleepTime);
290     }
291   }
292 
293   public static class RestartRsHoldingTable extends RestartActionBase {
294 
295     private final String tableName;
296 
297     public RestartRsHoldingTable(long sleepTime, String tableName) {
298       super(sleepTime);
299       this.tableName = tableName;
300     }
301 
302     @Override
303     public void perform() throws Exception {
304       HTable table = null;
305       Collection<ServerName> serverNames;
306       try {
307         Configuration conf = context.getHBaseIntegrationTestingUtility().getConfiguration();
308         table = new HTable(conf, tableName);
309         serverNames = table.getRegionLocations().values();
310       } catch (IOException e) {
311         LOG.debug("Error creating HTable used to get list of region locations.", e);
312         return;
313       } finally {
314         if (table != null) {
315           table.close();
316         }
317       }
318       Random random = new Random();
319       ServerName[] nameArray = serverNames.toArray(new ServerName[serverNames.size()]);
320       restartRs(nameArray[random.nextInt(nameArray.length)], sleepTime);
321     }
322   }
323 
324   public static class MoveRegionsOfTable extends Action {
325     private final long sleepTime;
326     private final byte[] tableNameBytes;
327 
328     public MoveRegionsOfTable(long sleepTime, String tableName) {
329       this.sleepTime = sleepTime;
330       this.tableNameBytes = Bytes.toBytes(tableName);
331     }
332 
333     @Override
334     public void perform() throws Exception {
335       try {
336         HBaseAdmin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
337         List<HRegionInfo> regions = admin.getTableRegions(tableNameBytes);
338         Collection<ServerName> serversList = admin.getClusterStatus().getServers();
339         ServerName[] servers = serversList.toArray(new ServerName[serversList.size()]);
340         Random random = new Random();
341         for (HRegionInfo regionInfo:regions) {
342           try {
343             byte[] destServerName =
344               Bytes.toBytes(servers[random.nextInt(servers.length)].getServerName());
345             admin.move(regionInfo.getRegionName(), destServerName);
346           } catch (Exception e) {
347             LOG.debug("Error moving region", e);
348           }
349         }
350         Thread.sleep(sleepTime);
351       } catch (Exception e) {
352         LOG.debug("Error performing MoveRegionsOfTable", e);
353       }
354     }
355   }
356 
357   /**
358    * Restarts a ratio of the running regionservers at the same time
359    */
360   public static class BatchRestartRs extends RestartActionBase {
361     float ratio; //ratio of regionservers to restart
362 
363     public BatchRestartRs(long sleepTime, float ratio) {
364       super(sleepTime);
365       this.ratio = ratio;
366     }
367 
368     @Override
369     public void perform() throws Exception {
370       LOG.info(String.format("Performing action: Batch restarting %d%% of region servers",
371           (int)(ratio * 100)));
372       List<ServerName> selectedServers = selectRandomItems(getCurrentServers(), ratio);
373 
374       for (ServerName server : selectedServers) {
375         LOG.info("Killing region server:" + server);
376         cluster.killRegionServer(server);
377       }
378 
379       for (ServerName server : selectedServers) {
380         cluster.waitForRegionServerToStop(server, TIMEOUT);
381       }
382 
383       LOG.info("Killed " + selectedServers.size() + " region servers. Reported num of rs:"
384           + cluster.getClusterStatus().getServersSize());
385 
386       sleep(sleepTime);
387 
388       for (ServerName server : selectedServers) {
389         LOG.info("Starting region server:" + server.getHostname());
390         cluster.startRegionServer(server.getHostname());
391 
392       }
393       for (ServerName server : selectedServers) {
394         cluster.waitForRegionServerToStart(server.getHostname(), TIMEOUT);
395       }
396       LOG.info("Started " + selectedServers.size() +" region servers. Reported num of rs:"
397           + cluster.getClusterStatus().getServersSize());
398     }
399   }
400 
401   /**
402    * Restarts a ratio of the regionservers in a rolling fashion. At each step, either kills a
403    * server, or starts one, sleeping randomly (0-sleepTime) in between steps.
404    */
405   public static class RollingBatchRestartRs extends BatchRestartRs {
406     public RollingBatchRestartRs(long sleepTime, float ratio) {
407       super(sleepTime, ratio);
408     }
409 
410     @Override
411     public void perform() throws Exception {
412       Random random = new Random();
413       LOG.info(String.format("Performing action: Rolling batch restarting %d%% of region servers",
414           (int)(ratio * 100)));
415       List<ServerName> selectedServers = selectRandomItems(getCurrentServers(), ratio);
416 
417       Queue<ServerName> serversToBeKilled = new LinkedList<ServerName>(selectedServers);
418       Queue<ServerName> deadServers = new LinkedList<ServerName>();
419 
420       //
421       while (!serversToBeKilled.isEmpty() || !deadServers.isEmpty()) {
422         boolean action = true; //action true = kill server, false = start server
423 
424         if (serversToBeKilled.isEmpty() || deadServers.isEmpty()) {
425           action = deadServers.isEmpty();
426         } else {
427           action = random.nextBoolean();
428         }
429 
430         if (action) {
431           ServerName server = serversToBeKilled.remove();
432           killRs(server);
433           deadServers.add(server);
434         } else {
435           ServerName server = deadServers.remove();
436           startRs(server);
437         }
438 
439         sleep(random.nextInt((int)sleepTime));
440       }
441     }
442   }
443 
444   public static class UnbalanceRegionsAction extends Action {
445     private double fractionOfRegions;
446     private double fractionOfServers;
447     private Random random = new Random();
448 
449     /**
450      * Unbalances the regions on the cluster by choosing "target" servers, and moving
451      * some regions from each of the non-target servers to random target servers.
452      * @param fractionOfRegions Fraction of regions to move from each server.
453      * @param fractionOfServers Fraction of servers to be chosen as targets.
454      */
455     public UnbalanceRegionsAction(double fractionOfRegions, double fractionOfServers) {
456       this.fractionOfRegions = fractionOfRegions;
457       this.fractionOfServers = fractionOfServers;
458     }
459 
460     @Override
461     public void perform() throws Exception {
462       LOG.info("Unbalancing regions");
463       ClusterStatus status = this.cluster.getClusterStatus();
464       List<ServerName> victimServers = new LinkedList<ServerName>(status.getServers());
465       int targetServerCount = (int)Math.ceil(fractionOfServers * victimServers.size());
466       List<byte[]> targetServers = new ArrayList<byte[]>(targetServerCount);
467       for (int i = 0; i < targetServerCount; ++i) {
468         int victimIx = random.nextInt(victimServers.size());
469         String serverName = victimServers.remove(victimIx).getServerName();
470         targetServers.add(Bytes.toBytes(serverName));
471       }
472 
473       List<byte[]> victimRegions = new LinkedList<byte[]>();
474       for (ServerName server : victimServers) {
475         HServerLoad serverLoad = status.getLoad(server);
476         // Ugh.
477         List<byte[]> regions = new LinkedList<byte[]>(serverLoad.getRegionsLoad().keySet());
478         int victimRegionCount = (int)Math.ceil(fractionOfRegions * regions.size());
479         LOG.debug("Removing " + victimRegionCount + " regions from " + server.getServerName());
480         for (int i = 0; i < victimRegionCount; ++i) {
481           int victimIx = random.nextInt(regions.size());
482           String regionId = HRegionInfo.encodeRegionName(regions.remove(victimIx));
483           victimRegions.add(Bytes.toBytes(regionId));
484         }
485       }
486 
487       LOG.info("Moving " + victimRegions.size() + " regions from " + victimServers.size()
488           + " servers to " + targetServers.size() + " different servers");
489       HBaseAdmin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
490       for (byte[] victimRegion : victimRegions) {
491         int targetIx = random.nextInt(targetServers.size());
492         admin.move(victimRegion, targetServers.get(targetIx));
493       }
494     }
495   }
496 
497   public static class ForceBalancerAction extends Action {
498     @Override
499     public void perform() throws Exception {
500       LOG.info("Balancing regions");
501       HBaseAdmin admin = this.context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
502       boolean result = admin.balancer();
503       if (!result) {
504         LOG.error("Balancer didn't succeed");
505       }
506     }
507   }
508 
509   /**
510    * A context for a Policy
511    */
512   public static class PolicyContext extends ActionContext {
513     public PolicyContext(IntegrationTestingUtility util) {
514       super(util);
515     }
516   }
517 
518   /**
519    * A policy to introduce chaos to the cluster
520    */
521   public static abstract class Policy extends StoppableImplementation implements Runnable {
522     protected PolicyContext context;
523     public void init(PolicyContext context) throws Exception {
524       this.context = context;
525     }
526   }
527 
528   /** A policy that runs multiple other policies one after the other */
529   public static class CompositeSequentialPolicy extends Policy {
530     private List<Policy> policies;
531     public CompositeSequentialPolicy(Policy... policies) {
532       this.policies = Arrays.asList(policies);
533     }
534 
535     @Override
536     public void stop(String why) {
537       super.stop(why);
538       for (Policy p : policies) {
539         p.stop(why);
540       }
541     }
542 
543     @Override
544     public void run() {
545       for (Policy p : policies) {
546         p.run();
547       }
548     }
549 
550     @Override
551     public void init(PolicyContext context) throws Exception {
552       super.init(context);
553       for (Policy p : policies) {
554         p.init(context);
555       }
556     }
557   }
558 
559   /** A policy which does stuff every time interval. */
560   public static abstract class PeriodicPolicy extends Policy {
561     private long periodMs;
562 
563     public PeriodicPolicy(long periodMs) {
564       this.periodMs = periodMs;
565     }
566 
567     @Override
568     public void run() {
569       // Add some jitter.
570       int jitter = new Random().nextInt((int)periodMs);
571       LOG.info("Sleeping for " + jitter + " to add jitter");
572       Threads.sleep(jitter);
573 
574       while (!isStopped()) {
575         long start = System.currentTimeMillis();
576         runOneIteration();
577 
578         if (isStopped()) return;
579         long sleepTime = periodMs - (System.currentTimeMillis() - start);
580         if (sleepTime > 0) {
581           LOG.info("Sleeping for: " + sleepTime);
582           Threads.sleep(sleepTime);
583         }
584       }
585     }
586 
587     protected abstract void runOneIteration();
588 
589     @Override
590     public void init(PolicyContext context) throws Exception {
591       super.init(context);
592       LOG.info("Using ChaosMonkey Policy: " + this.getClass() + ", period: " + periodMs);
593     }
594   }
595 
596   /** A policy which performs a sequence of actions deterministically. */
597   public static class DoActionsOncePolicy extends PeriodicPolicy {
598     private List<Action> actions;
599 
600     public DoActionsOncePolicy(long periodMs, List<Action> actions) {
601       super(periodMs);
602       this.actions = new ArrayList<ChaosMonkey.Action>(actions);
603     }
604 
605     public DoActionsOncePolicy(long periodMs, Action... actions) {
606       this(periodMs, Arrays.asList(actions));
607     }
608 
609     @Override
610     protected void runOneIteration() {
611       if (actions.isEmpty()) {
612         this.stop("done");
613         return;
614       }
615       Action action = actions.remove(0);
616 
617       try {
618         action.perform();
619       } catch (Exception ex) {
620         LOG.warn("Exception occured during performing action: "
621             + StringUtils.stringifyException(ex));
622       }
623     }
624 
625     @Override
626     public void init(PolicyContext context) throws Exception {
627       super.init(context);
628       for (Action action : actions) {
629         action.init(this.context);
630       }
631     }
632   }
633 
634   /**
635    * A policy, which picks a random action according to the given weights,
636    * and performs it every configurable period.
637    */
638   public static class PeriodicRandomActionPolicy extends PeriodicPolicy {
639     private List<Pair<Action, Integer>> actions;
640 
641     public PeriodicRandomActionPolicy(long periodMs, List<Pair<Action, Integer>> actions) {
642       super(periodMs);
643       this.actions = actions;
644     }
645 
646     public PeriodicRandomActionPolicy(long periodMs, Pair<Action, Integer>... actions) {
647       // We don't expect it to be modified.
648       this(periodMs, Arrays.asList(actions));
649     }
650 
651     public PeriodicRandomActionPolicy(long periodMs, Action... actions) {
652       super(periodMs);
653       this.actions = new ArrayList<Pair<Action, Integer>>(actions.length);
654       for (Action action : actions) {
655         this.actions.add(new Pair<Action, Integer>(action, 1));
656       }
657     }
658 
659     @Override
660     protected void runOneIteration() {
661       Action action = selectWeightedRandomItem(actions);
662       try {
663         action.perform();
664       } catch (Exception ex) {
665         LOG.warn("Exception occured during performing action: "
666             + StringUtils.stringifyException(ex));
667       }
668     }
669 
670     @Override
671     public void init(PolicyContext context) throws Exception {
672       super.init(context);
673       for (Pair<Action, Integer> action : actions) {
674         action.getFirst().init(this.context);
675       }
676     }
677   }
678 
679   /** Selects a random item from the given items */
680   static <T> T selectRandomItem(T[] items) {
681     Random random = new Random();
682     return items[random.nextInt(items.length)];
683   }
684 
685   /** Selects a random item from the given items with weights*/
686   static <T> T selectWeightedRandomItem(List<Pair<T, Integer>> items) {
687     Random random = new Random();
688     int totalWeight = 0;
689     for (Pair<T, Integer> pair : items) {
690       totalWeight += pair.getSecond();
691     }
692 
693     int cutoff = random.nextInt(totalWeight);
694     int cummulative = 0;
695     T item = null;
696 
697     //warn: O(n)
698     for (int i=0; i<items.size(); i++) {
699       int curWeight = items.get(i).getSecond();
700       if ( cutoff < cummulative + curWeight) {
701         item = items.get(i).getFirst();
702         break;
703       }
704       cummulative += curWeight;
705     }
706 
707     return item;
708   }
709 
710   /** Selects and returns ceil(ratio * items.length) random items from the given array */
711   static <T> List<T> selectRandomItems(T[] items, float ratio) {
712     Random random = new Random();
713     int remaining = (int)Math.ceil(items.length * ratio);
714 
715     List<T> selectedItems = new ArrayList<T>(remaining);
716 
717     for (int i=0; i<items.length && remaining > 0; i++) {
718       if (random.nextFloat() < ((float)remaining/(items.length-i))) {
719         selectedItems.add(items[i]);
720         remaining--;
721       }
722     }
723 
724     return selectedItems;
725   }
726 
727   /**
728    * All actions that deal with RS's with the following weights (relative probabilities):
729    *  - Restart active master (sleep 5 sec)                    : 2
730    *  - Restart random regionserver (sleep 5 sec)              : 2
731    *  - Restart random regionserver (sleep 60 sec)             : 2
732    *  - Restart META regionserver (sleep 5 sec)                : 1
733    *  - Restart ROOT regionserver (sleep 5 sec)                : 1
734    *  - Batch restart of 50% of regionservers (sleep 5 sec)    : 2
735    *  - Rolling restart of 100% of regionservers (sleep 5 sec) : 2
736    */
737   @SuppressWarnings("unchecked")
738   private static final List<Pair<Action, Integer>> ALL_ACTIONS = Lists.newArrayList(
739       new Pair<Action,Integer>(new RestartActiveMaster(FIVE_SEC), 2),
740       new Pair<Action,Integer>(new RestartRandomRs(FIVE_SEC), 2),
741       new Pair<Action,Integer>(new RestartRandomRs(ONE_MIN), 2),
742       new Pair<Action,Integer>(new RestartRsHoldingMeta(FIVE_SEC), 1),
743       new Pair<Action,Integer>(new RestartRsHoldingRoot(FIVE_SEC), 1),
744       new Pair<Action,Integer>(new BatchRestartRs(FIVE_SEC, 0.5f), 2),
745       new Pair<Action,Integer>(new RollingBatchRestartRs(FIVE_SEC, 1.0f), 2)
746   );
747 
748   public static final String EVERY_MINUTE_RANDOM_ACTION_POLICY = "EVERY_MINUTE_RANDOM_ACTION_POLICY";
749 
750   private Policy[] policies;
751   private Thread[] monkeyThreads;
752 
753   public void start() throws Exception {
754     monkeyThreads = new Thread[policies.length];
755 
756     for (int i=0; i<policies.length; i++) {
757       policies[i].init(new PolicyContext(this.util));
758       Thread monkeyThread = new Thread(policies[i]);
759       monkeyThread.start();
760       monkeyThreads[i] = monkeyThread;
761     }
762   }
763 
764   @Override
765   public void stop(String why) {
766     for (Policy policy : policies) {
767       policy.stop(why);
768     }
769   }
770 
771   @Override
772   public boolean isStopped() {
773     return policies[0].isStopped();
774   }
775 
776   /**
777    * Wait for ChaosMonkey to stop.
778    * @throws InterruptedException
779    */
780   public void waitForStop() throws InterruptedException {
781     for (Thread monkeyThread : monkeyThreads) {
782       monkeyThread.join();
783     }
784   }
785 
786   private static final Map<String, Policy> NAMED_POLICIES = Maps.newHashMap();
787   static {
788     NAMED_POLICIES.put(EVERY_MINUTE_RANDOM_ACTION_POLICY,
789         new PeriodicRandomActionPolicy(ONE_MIN, ALL_ACTIONS));
790   }
791 
792   @Override
793   protected void addOptions() {
794     addOptWithArg("policy", "a named policy defined in ChaosMonkey.java. Possible values: "
795         + NAMED_POLICIES.keySet());
796     //we can add more options, and make policies more configurable
797   }
798 
799   @Override
800   protected void processOptions(CommandLine cmd) {
801     String[] policies = cmd.getOptionValues("policy");
802     if (policies != null) {
803       setPoliciesByName(policies);
804     } else {
805       // Set a default policy if none is provided
806       setPolicies(NAMED_POLICIES.get(EVERY_MINUTE_RANDOM_ACTION_POLICY));
807     }
808   }
809 
810   @Override
811   protected int doWork() throws Exception {
812     start();
813     waitForStop();
814     return 0;
815   }
816 
817   public static void main(String[] args) throws Exception {
818     Configuration conf = HBaseConfiguration.create();
819     IntegrationTestingUtility.setUseDistributedCluster(conf);
820     IntegrationTestingUtility util = new IntegrationTestingUtility(conf);
821     util.initializeCluster(1);
822 
823     ChaosMonkey monkey = new ChaosMonkey(util);
824     int ret = ToolRunner.run(conf, monkey, args);
825     System.exit(ret);
826   }
827 
828 }