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.mttr;
20  
21  import com.google.common.base.Objects;
22  import org.apache.commons.lang.RandomStringUtils;
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.commons.math.stat.descriptive.DescriptiveStatistics;
26  import org.apache.hadoop.hbase.ClusterStatus;
27  import org.apache.hadoop.hbase.HColumnDescriptor;
28  import org.apache.hadoop.hbase.HTableDescriptor;
29  import org.apache.hadoop.hbase.IntegrationTestingUtility;
30  import org.apache.hadoop.hbase.IntegrationTests;
31  import org.apache.hadoop.hbase.client.HBaseAdmin;
32  import org.apache.hadoop.hbase.client.HTable;
33  import org.apache.hadoop.hbase.client.Put;
34  import org.apache.hadoop.hbase.client.Result;
35  import org.apache.hadoop.hbase.client.ResultScanner;
36  import org.apache.hadoop.hbase.client.Scan;
37  import org.apache.hadoop.hbase.filter.KeyOnlyFilter;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.ChaosMonkey;
40  import org.apache.hadoop.hbase.util.LoadTestTool;
41  import org.junit.AfterClass;
42  import org.junit.BeforeClass;
43  import org.junit.Test;
44  import org.junit.experimental.categories.Category;
45  
46  import java.io.IOException;
47  import java.util.ArrayList;
48  import java.util.concurrent.Callable;
49  import java.util.concurrent.ExecutorService;
50  import java.util.concurrent.Executors;
51  import java.util.concurrent.Future;
52  import java.util.concurrent.TimeUnit;
53  
54  import static junit.framework.Assert.assertEquals;
55  
56  /**
57   * Integration test that should benchmark how fast HBase can recover from failures. This test starts
58   * different threads:
59   * <ol>
60   * <li>
61   * Load Test Tool.<br/>
62   * This runs so that all RegionServers will have some load and HLogs will be full.
63   * </li>
64   * <li>
65   * Scan thread.<br/>
66   * This thread runs a very short scan over and over again recording how log it takes to respond.
67   * The longest response is assumed to be the time it took to recover.
68   * </li>
69   * <li>
70   * Put thread.<br/>
71   * This thread just like the scan thread except it does a very small put.
72   * </li>
73   * <li>
74   * Admin thread. <br/>
75   * This thread will continually go to the master to try and get the cluster status.  Just like the
76   * put and scan threads, the time to respond is recorded.
77   * </li>
78   * <li>
79   * Chaos Monkey thread.<br/>
80   * This thread runs a ChaosMonkey.Action.
81   * </li>
82   * </ol>
83   * <p/>
84   * The ChaosMonkey actions currently run are:
85   * <ul>
86   * <li>Restart the RegionServer holding meta.</li>
87   * <li>Restart the RegionServer holding the table the scan and put threads are targeting.</li>
88   * <li>Move the Regions of the table used by the scan and put threads.</li>
89   * <li>Restart the master.</li>
90   * </ul>
91   * <p/>
92   * At the end of the test a log line is output on the INFO level containing the timing data that was
93   * collected.
94   */
95  
96  @Category(IntegrationTests.class)
97  public class IntegrationTestMTTR {
98    /**
99     * Constants.
100    */
101   private static final byte[] FAMILY = Bytes.toBytes("d");
102   private static final Log LOG = LogFactory.getLog(IntegrationTestMTTR.class);
103   private static final long SLEEP_TIME = 60 * 1000l;
104 
105   /**
106    * Configurable table names.
107    */
108   private static String tableName;
109   private static byte[] tableNameBytes;
110   private static String loadTableName;
111   private static byte[] loadTableNameBytes;
112 
113   /**
114    * Util to get at the cluster.
115    */
116   private static IntegrationTestingUtility util;
117 
118   /**
119    * Executor for test threads.
120    */
121   private static ExecutorService executorService;
122 
123   /**
124    * All of the chaos monkey actions used.
125    */
126   private static ChaosMonkey.Action restartRSAction;
127   private static ChaosMonkey.Action restartMetaAction;
128   private static ChaosMonkey.Action moveRegionAction;
129   private static ChaosMonkey.Action restartMasterAction;
130 
131   /**
132    * The load test tool used to create load and make sure that HLogs aren't empty.
133    */
134   private static LoadTestTool loadTool;
135 
136 
137   @BeforeClass
138   public static void setUp() throws Exception {
139     // Set up the integration test util
140     if (util == null) {
141       util = new IntegrationTestingUtility();
142     }
143 
144     // Make sure there are three servers.
145     util.initializeCluster(3);
146 
147     // Set up the load test tool.
148     loadTool = new LoadTestTool();
149     loadTool.setConf(util.getConfiguration());
150 
151     // Create executor with enough threads to restart rs's,
152     // run scans, puts, admin ops and load test tool.
153     executorService = Executors.newFixedThreadPool(8);
154 
155     // Set up the tables needed.
156     setupTables();
157 
158     // Set up the actions.
159     setupActions();
160   }
161 
162   private static void setupActions() throws IOException {
163     // Set up the action that will restart a region server holding a region from our table
164     // because this table should only have one region we should be good.
165     restartRSAction = new ChaosMonkey.RestartRsHoldingTable(SLEEP_TIME, tableName);
166 
167     // Set up the action that will kill the region holding meta.
168     restartMetaAction = new ChaosMonkey.RestartRsHoldingMeta(SLEEP_TIME);
169 
170     // Set up the action that will move the regions of our table.
171     moveRegionAction = new ChaosMonkey.MoveRegionsOfTable(SLEEP_TIME, tableName);
172 
173     // Kill the master
174     restartMasterAction = new ChaosMonkey.RestartActiveMaster(1000);
175 
176     // Give the action the access to the cluster.
177     ChaosMonkey.ActionContext actionContext = new ChaosMonkey.ActionContext(util);
178     restartRSAction.init(actionContext);
179     restartMetaAction.init(actionContext);
180     moveRegionAction.init(actionContext);
181     restartMasterAction.init(actionContext);
182   }
183 
184   private static void setupTables() throws IOException {
185     // Get the table name.
186     tableName = util.getConfiguration()
187         .get("hbase.IntegrationTestMTTR.tableName", "IntegrationTestMTTR");
188     tableNameBytes = Bytes.toBytes(tableName);
189 
190     loadTableName = util.getConfiguration()
191         .get("hbase.IntegrationTestMTTR.loadTableName", "IntegrationTestMTTRLoadTestTool");
192     loadTableNameBytes = Bytes.toBytes(loadTableName);
193 
194     if (util.getHBaseAdmin().tableExists(tableNameBytes)) {
195       util.deleteTable(tableNameBytes);
196     }
197 
198     if (util.getHBaseAdmin().tableExists(loadTableName)) {
199       util.deleteTable(loadTableNameBytes);
200     }
201 
202     // Create the table.  If this fails then fail everything.
203     HTableDescriptor tableDescriptor = new HTableDescriptor(tableNameBytes);
204 
205     // Make the max file size huge so that splits don't happen during the test.
206     tableDescriptor.setMaxFileSize(Long.MAX_VALUE);
207 
208     HColumnDescriptor descriptor = new HColumnDescriptor(FAMILY);
209     descriptor.setMaxVersions(1);
210     tableDescriptor.addFamily(descriptor);
211     util.getHBaseAdmin().createTable(tableDescriptor);
212 
213     // Setup the table for LoadTestTool
214     int ret = loadTool.run(new String[]{"-tn", loadTableName, "-init_only"});
215     assertEquals("Failed to initialize LoadTestTool", 0, ret);
216   }
217 
218   @AfterClass
219   public static void after() throws IOException {
220     // Clean everything up.
221     util.restoreCluster();
222     util = null;
223 
224     // Stop the threads so that we know everything is complete.
225     executorService.shutdown();
226     executorService = null;
227 
228     // Clean up the actions.
229     moveRegionAction = null;
230     restartMetaAction = null;
231     restartRSAction = null;
232     restartMasterAction = null;
233 
234     loadTool = null;
235   }
236 
237   @Test
238   public void testRestartRsHoldingTable() throws Exception {
239     run(new ActionCallable(restartRSAction), "RestartRsHoldingTable");
240   }
241 
242   @Test
243   public void testKillRsHoldingMeta() throws Exception {
244     run(new ActionCallable(restartMetaAction), "KillRsHoldingMeta");
245   }
246 
247   @Test
248   public void testMoveRegion() throws Exception {
249     run(new ActionCallable(moveRegionAction), "MoveRegion");
250   }
251 
252   @Test
253   public void testRestartMaster() throws Exception {
254     run(new ActionCallable(restartMasterAction), "RestartMaster");
255   }
256 
257   public void run(Callable<Boolean> monkeyCallable, String testName) throws Exception {
258     int maxIters = util.getHBaseClusterInterface().isDistributedCluster() ? 10 : 3;
259 
260     // Array to keep track of times.
261     ArrayList<TimingResult> resultPuts = new ArrayList<TimingResult>(maxIters);
262     ArrayList<TimingResult> resultScan = new ArrayList<TimingResult>(maxIters);
263     ArrayList<TimingResult> resultAdmin = new ArrayList<TimingResult>(maxIters);
264     long start = System.nanoTime();
265 
266     // We're going to try this multiple times
267     for (int fullIterations = 0; fullIterations < maxIters; fullIterations++) {
268       // Create and start executing a callable that will kill the servers
269       Future<Boolean> monkeyFuture = executorService.submit(monkeyCallable);
270 
271       // Pass that future to the timing Callables.
272       Future<TimingResult> putFuture = executorService.submit(new PutCallable(monkeyFuture));
273       Future<TimingResult> scanFuture = executorService.submit(new ScanCallable(monkeyFuture));
274       Future<TimingResult> adminFuture = executorService.submit(new AdminCallable(monkeyFuture));
275 
276       Future<Boolean> loadFuture = executorService.submit(new LoadCallable(monkeyFuture));
277 
278       monkeyFuture.get();
279       loadFuture.get();
280 
281       // Get the values from the futures.
282       TimingResult putTime = putFuture.get();
283       TimingResult scanTime = scanFuture.get();
284       TimingResult adminTime = adminFuture.get();
285 
286       // Store the times to display later.
287       resultPuts.add(putTime);
288       resultScan.add(scanTime);
289       resultAdmin.add(adminTime);
290 
291       // Wait some time for everything to settle down.
292       Thread.sleep(5000l);
293     }
294 
295     long runtimeMs = TimeUnit.MILLISECONDS.convert(System.nanoTime() - start, TimeUnit.NANOSECONDS);
296 
297     Objects.ToStringHelper helper = Objects.toStringHelper("MTTRResults")
298         .add("putResults", resultPuts)
299         .add("scanResults", resultScan)
300         .add("adminResults", resultAdmin)
301         .add("totalRuntimeMs", runtimeMs)
302         .add("name", testName);
303 
304     // Log the info
305     LOG.info(helper.toString());
306   }
307 
308   /**
309    * Class to store results of TimingCallable.
310    *
311    * Stores times and trace id.
312    */
313   private class TimingResult {
314     DescriptiveStatistics stats = new DescriptiveStatistics();
315 
316     /**
317      * Add a result to this aggregate result.
318      * @param time Time in nanoseconds
319      * @param span Span.  To be kept if the time taken was over 1 second
320      */
321     public void addResult(long time) {
322       stats.addValue(TimeUnit.MILLISECONDS.convert(time, TimeUnit.NANOSECONDS));
323     }
324 
325     public String toString() {
326       Objects.ToStringHelper helper = Objects.toStringHelper(this)
327           .add("numResults", stats.getN())
328           .add("minTime", stats.getMin())
329           .add("meanTime", stats.getMean())
330           .add("maxTime", stats.getMax())
331           .add("25th", stats.getPercentile(25))
332           .add("50th", stats.getPercentile(50))
333           .add("75th", stats.getPercentile(75))
334           .add("90th", stats.getPercentile(90))
335           .add("95th", stats.getPercentile(95))
336           .add("99th", stats.getPercentile(99))
337           .add("99.9th", stats.getPercentile(99.9))
338           .add("99.99th", stats.getPercentile(99.99));
339       return helper.toString();
340     }
341   }
342 
343   /**
344    * Base class for actions that need to record the time needed to recover from a failure.
345    */
346   public abstract class TimingCallable implements Callable<TimingResult> {
347     protected final Future future;
348 
349     public TimingCallable(Future f) {
350       future = f;
351     }
352 
353     @Override
354     public TimingResult call() throws Exception {
355       TimingResult result = new TimingResult();
356       int numAfterDone = 0;
357       // Keep trying until the rs is back up and we've gotten a put through
358       while (numAfterDone < 10) {
359         long start = System.nanoTime();
360         try {
361           boolean actionResult = doAction();
362           if (actionResult && future.isDone()) {
363             numAfterDone ++;
364           }
365         } catch (Exception e) {
366           numAfterDone = 0;
367         }
368         result.addResult(System.nanoTime() - start);
369       }
370       return result;
371     }
372 
373     protected abstract boolean doAction() throws Exception;
374 
375     protected String getSpanName() {
376       return this.getClass().getSimpleName();
377     }
378   }
379 
380   /**
381    * Callable that will keep putting small amounts of data into a table
382    * until  the future supplied returns.  It keeps track of the max time.
383    */
384   public class PutCallable extends TimingCallable {
385 
386     private final HTable table;
387 
388     public PutCallable(Future f) throws IOException {
389       super(f);
390       this.table = new HTable(util.getConfiguration(), tableNameBytes);
391     }
392 
393     @Override
394     protected boolean doAction() throws Exception {
395       Put p = new Put(Bytes.toBytes(RandomStringUtils.randomAlphanumeric(5)));
396       p.add(FAMILY, Bytes.toBytes("\0"), Bytes.toBytes(RandomStringUtils.randomAscii(5)));
397       table.put(p);
398       table.flushCommits();
399       return true;
400     }
401 
402     @Override
403     protected String getSpanName() {
404       return "MTTR Put Test";
405     }
406   }
407 
408   /**
409    * Callable that will keep scanning for small amounts of data until the
410    * supplied future returns.  Returns the max time taken to scan.
411    */
412   public class ScanCallable extends TimingCallable {
413     private final HTable table;
414 
415     public ScanCallable(Future f) throws IOException {
416       super(f);
417       this.table = new HTable(util.getConfiguration(), tableNameBytes);
418     }
419 
420     @Override
421     protected boolean doAction() throws Exception {
422       ResultScanner rs = null;
423       try {
424       Scan s = new Scan();
425       s.setBatch(2);
426       s.addFamily(FAMILY);
427       s.setFilter(new KeyOnlyFilter());
428       s.setMaxVersions(1);
429 
430       rs = table.getScanner(s);
431       Result result = rs.next();
432       return rs != null && result != null && result.size() > 0;
433       } finally {
434         if (rs != null) {
435           rs.close();
436         }
437       }
438     }
439     @Override
440     protected String getSpanName() {
441       return "MTTR Scan Test";
442     }
443   }
444 
445   /**
446    * Callable that will keep going to the master for cluster status.  Returns the max time taken.
447    */
448   public class AdminCallable extends TimingCallable {
449 
450     public AdminCallable(Future f) throws IOException {
451       super(f);
452     }
453 
454     @Override
455     protected boolean doAction() throws Exception {
456       HBaseAdmin admin = new HBaseAdmin(util.getConfiguration());
457       ClusterStatus status = admin.getClusterStatus();
458       return status != null;
459     }
460 
461     @Override
462     protected String getSpanName() {
463       return "MTTR Admin Test";
464     }
465   }
466 
467 
468   public class ActionCallable implements Callable<Boolean> {
469     private final ChaosMonkey.Action action;
470 
471     public ActionCallable(ChaosMonkey.Action action) {
472       this.action = action;
473     }
474 
475     @Override
476     public Boolean call() throws Exception {
477       this.action.perform();
478       return true;
479     }
480   }
481 
482   /**
483    * Callable used to make sure the cluster has some load on it.
484    * This callable uses LoadTest tool to
485    */
486   public class LoadCallable implements Callable<Boolean> {
487 
488     private final Future future;
489 
490     public LoadCallable(Future f) {
491       future = f;
492     }
493 
494     @Override
495     public Boolean call() throws Exception {
496       int colsPerKey = 10;
497       int recordSize = 500;
498       int numServers = util.getHBaseClusterInterface().getInitialClusterStatus().getServersSize();
499       int numKeys = numServers * 5000;
500       int writeThreads = 10;
501 
502 
503       // Loop until the chaos monkey future is done.
504       // But always go in just in case some action completes quickly
505       do {
506         int ret = loadTool.run(new String[]{
507             "-tn", loadTableName,
508             "-write", String.format("%d:%d:%d", colsPerKey, recordSize, writeThreads),
509             "-num_keys", String.valueOf(numKeys),
510             "-skip_init"
511         });
512         assertEquals("Load failed", 0, ret);
513       } while (!future.isDone());
514 
515       return true;
516     }
517   }
518 }