1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_mgr_wait_for_zk_delete;
23  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_final_transistion_failed;
24  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_preempt_task;
25  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_acquired;
26  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_done;
27  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_err;
28  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.tot_wkr_task_resigned;
29  import static org.junit.Assert.assertEquals;
30  import static org.junit.Assert.assertTrue;
31  import static org.junit.Assert.fail;
32  
33  import java.io.IOException;
34  import java.util.Arrays;
35  import java.util.Iterator;
36  import java.util.List;
37  import java.util.NavigableSet;
38  import java.util.TreeSet;
39  import java.util.concurrent.ExecutorService;
40  import java.util.concurrent.Executors;
41  import java.util.concurrent.Future;
42  import java.util.concurrent.TimeUnit;
43  import java.util.concurrent.TimeoutException;
44  import java.util.concurrent.atomic.AtomicLong;
45  
46  import org.apache.commons.logging.Log;
47  import org.apache.commons.logging.LogFactory;
48  import org.apache.hadoop.conf.Configuration;
49  import org.apache.hadoop.fs.FSDataOutputStream;
50  import org.apache.hadoop.fs.FileStatus;
51  import org.apache.hadoop.fs.FileSystem;
52  import org.apache.hadoop.fs.Path;
53  import org.apache.hadoop.hbase.HBaseConfiguration;
54  import org.apache.hadoop.hbase.HBaseTestingUtility;
55  import org.apache.hadoop.hbase.HRegionInfo;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.KeyValue;
58  import org.apache.hadoop.hbase.LargeTests;
59  import org.apache.hadoop.hbase.MiniHBaseCluster;
60  import org.apache.hadoop.hbase.client.HTable;
61  import org.apache.hadoop.hbase.client.Put;
62  import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
63  import org.apache.hadoop.hbase.regionserver.HRegion;
64  import org.apache.hadoop.hbase.regionserver.HRegionServer;
65  import org.apache.hadoop.hbase.regionserver.wal.HLog;
66  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
67  import org.apache.hadoop.hbase.util.Bytes;
68  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
69  import org.apache.hadoop.hbase.util.FSUtils;
70  import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
71  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
72  import org.apache.hadoop.hbase.util.Threads;
73  import org.apache.hadoop.hbase.zookeeper.MiniZooKeeperCluster;
74  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
75  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
76  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
77  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
78  import org.apache.hadoop.hdfs.MiniDFSCluster;
79  import org.apache.log4j.Level;
80  import org.apache.log4j.Logger;
81  import org.apache.zookeeper.KeeperException;
82  import org.junit.After;
83  import org.junit.AfterClass;
84  import org.junit.Before;
85  import org.junit.BeforeClass;
86  import org.junit.Test;
87  import org.junit.experimental.categories.Category;
88  
89  @Category(LargeTests.class)
90  public class TestDistributedLogSplitting {
91    private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
92    static {
93      Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
94    }
95  
96    // Start a cluster with 2 masters and 3 regionservers
97    final int NUM_MASTERS = 2;
98    final int NUM_RS = 6;
99  
100   MiniHBaseCluster cluster;
101   HMaster master;
102   Configuration conf;
103   static HBaseTestingUtility TEST_UTIL;
104   static Configuration originalConf;
105   static MiniDFSCluster dfsCluster;
106   static MiniZooKeeperCluster zkCluster;
107 
108   @BeforeClass
109   public static void setup() throws Exception {
110     TEST_UTIL = new HBaseTestingUtility(HBaseConfiguration.create());
111     dfsCluster = TEST_UTIL.startMiniDFSCluster(1);
112     zkCluster = TEST_UTIL.startMiniZKCluster();
113     originalConf = TEST_UTIL.getConfiguration();
114   }
115 
116   @AfterClass
117   public static void tearDown() throws Exception {
118     TEST_UTIL.shutdownMiniZKCluster();
119     TEST_UTIL.shutdownMiniDFSCluster();
120     TEST_UTIL.shutdownMiniHBaseCluster();
121   }
122 
123   private void startCluster(int num_rs) throws Exception{
124     conf = HBaseConfiguration.create();
125     startCluster(NUM_MASTERS, num_rs, conf);
126   }
127 
128   private void startCluster(int num_master, int num_rs, Configuration inConf) throws Exception {
129     ZKSplitLog.Counters.resetCounters();
130     LOG.info("Starting cluster");
131     this.conf = inConf;
132     conf.getLong("hbase.splitlog.max.resubmit", 0);
133     // Make the failure test faster
134     conf.setInt("zookeeper.recovery.retry", 0);
135     TEST_UTIL.shutdownMiniHBaseCluster();
136     TEST_UTIL = new HBaseTestingUtility(conf);
137     TEST_UTIL.setDFSCluster(dfsCluster);
138     TEST_UTIL.setZkCluster(zkCluster);
139     TEST_UTIL.startMiniHBaseCluster(NUM_MASTERS, num_rs);
140     cluster = TEST_UTIL.getHBaseCluster();
141     LOG.info("Waiting for active/ready master");
142     cluster.waitForActiveAndReadyMaster();
143     master = cluster.getMaster();
144     while (cluster.getLiveRegionServerThreads().size() < num_rs) {
145       Threads.sleep(1);
146     }
147   }
148 
149   @Before
150   public void before() throws Exception {
151     // refresh configuration
152     conf = HBaseConfiguration.create(originalConf);
153   }
154   
155   @After
156   public void after() throws Exception {
157     if (TEST_UTIL.getHBaseCluster() != null) {
158       for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
159         mt.getMaster().abort("closing...", new Exception("Trace info"));
160       }
161     }
162     TEST_UTIL.shutdownMiniHBaseCluster();
163     TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
164     ZKUtil.deleteNodeRecursively(HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL), "/hbase");
165   }
166 
167   @Test (timeout=300000)
168   public void testThreeRSAbort() throws Exception {
169     LOG.info("testThreeRSAbort");
170     final int NUM_REGIONS_TO_CREATE = 40;
171     final int NUM_ROWS_PER_REGION = 100;
172 
173     startCluster(NUM_RS); // NUM_RS=6.
174 
175     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
176         "distributed log splitting test", null);
177 
178     HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
179     populateDataInTable(NUM_ROWS_PER_REGION, "family");
180 
181 
182     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
183     assertEquals(NUM_RS, rsts.size());
184     rsts.get(0).getRegionServer().abort("testing");
185     rsts.get(1).getRegionServer().abort("testing");
186     rsts.get(2).getRegionServer().abort("testing");
187 
188     long start = EnvironmentEdgeManager.currentTimeMillis();
189     while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
190       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
191         assertTrue(false);
192       }
193       Thread.sleep(200);
194     }
195 
196     start = EnvironmentEdgeManager.currentTimeMillis();
197     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
198       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
199         assertTrue(false);
200       }
201       Thread.sleep(200);
202     }
203 
204     assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
205         TEST_UTIL.countRows(ht));
206     ht.close();
207   }
208 
209   @Test (timeout=300000)
210   public void testRecoveredEdits() throws Exception {
211     LOG.info("testRecoveredEdits");
212     startCluster(NUM_RS);
213     final int NUM_LOG_LINES = 1000;
214     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
215     // turn off load balancing to prevent regions from moving around otherwise
216     // they will consume recovered.edits
217     master.balanceSwitch(false);
218     FileSystem fs = master.getMasterFileSystem().getFileSystem();
219 
220     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
221     
222     Path rootdir = FSUtils.getRootDir(conf);
223 
224     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
225         "table", "family", 40);
226     byte[] table = Bytes.toBytes("table");
227     List<HRegionInfo> regions = null;
228     HRegionServer hrs = null;
229     for (int i = 0; i < NUM_RS; i++) {
230       hrs = rsts.get(i).getRegionServer();
231       regions = hrs.getOnlineRegions();
232       if (regions.size() != 0) break;
233     }
234     final Path logDir = new Path(rootdir, HLog.getHLogDirectoryName(hrs
235         .getServerName().toString()));
236     
237     LOG.info("#regions = " + regions.size());
238     Iterator<HRegionInfo> it = regions.iterator();
239     while (it.hasNext()) {
240       HRegionInfo region = it.next();
241       if (region.isMetaTable()) {
242         it.remove();
243       }
244     }
245     makeHLog(hrs.getWAL(), regions, "table",
246         NUM_LOG_LINES, 100);
247 
248     slm.splitLogDistributed(logDir);
249 
250     int count = 0;
251     for (HRegionInfo hri : regions) {
252 
253       Path tdir = HTableDescriptor.getTableDir(rootdir, table);
254       Path editsdir =
255         HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
256         hri.getEncodedName()));
257       LOG.debug("checking edits dir " + editsdir);
258       FileStatus[] files = fs.listStatus(editsdir);
259       assertEquals(1, files.length);
260       int c = countHLog(files[0].getPath(), fs, conf);
261       count += c;
262       LOG.info(c + " edits in " + files[0].getPath());
263     }
264     assertEquals(NUM_LOG_LINES, count);
265   }
266 
267   @Test(timeout = 300000)
268   public void testMasterStartsUpWithLogSplittingWork() throws Exception {
269     LOG.info("testMasterStartsUpWithLogSplittingWork");
270     Configuration curConf = HBaseConfiguration.create();
271     curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
272     startCluster(2, NUM_RS, curConf);
273 
274     final int NUM_REGIONS_TO_CREATE = 40;
275     final int NUM_LOG_LINES = 1000;
276     // turn off load balancing to prevent regions from moving around otherwise
277     // they will consume recovered.edits
278     master.balanceSwitch(false);
279 
280     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
281     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
282     HTable ht = installTable(zkw, "table", "f", NUM_REGIONS_TO_CREATE);
283 
284     List<HRegionInfo> regions = null;
285     HRegionServer hrs = null;
286     for (int i = 0; i < NUM_RS; i++) {
287       boolean isCarryingMeta = false;
288       hrs = rsts.get(i).getRegionServer();
289       regions = hrs.getOnlineRegions();
290       for (HRegionInfo region : regions) {
291         if (region.isRootRegion() || region.isMetaRegion()) {
292           isCarryingMeta = true;
293           break;
294         }
295       }
296       if (isCarryingMeta) {
297         continue;
298       }
299       break;
300     }
301 
302     LOG.info("#regions = " + regions.size());
303     Iterator<HRegionInfo> it = regions.iterator();
304     while (it.hasNext()) {
305       HRegionInfo region = it.next();
306       if (region.isMetaTable()) {
307         it.remove();
308       }
309     }
310     makeHLog(hrs.getWAL(), regions, "table", NUM_LOG_LINES, 100);
311 
312     // abort master
313     abortMaster(cluster);
314 
315     // abort RS
316     int numRS = cluster.getLiveRegionServerThreads().size();
317     LOG.info("Aborting region server: " + hrs.getServerName());
318     hrs.abort("testing");
319 
320     // wait for the RS dies
321     long start = EnvironmentEdgeManager.currentTimeMillis();
322     while (cluster.getLiveRegionServerThreads().size() > (numRS - 1)) {
323       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
324         assertTrue(false);
325       }
326       Thread.sleep(200);
327     }
328 
329     Thread.sleep(2000);
330     LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
331     
332     startMasterTillNoDeadServers(cluster);
333     
334     start = EnvironmentEdgeManager.currentTimeMillis();
335     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
336       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
337         assertTrue("Timedout", false);
338       }
339       Thread.sleep(200);
340     }
341 
342     LOG.info("Current Open Regions After Master Node Starts Up:"
343         + getAllOnlineRegions(cluster).size());
344 
345     assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
346 
347     ht.close();
348   }
349 
350   /**
351    * The original intention of this test was to force an abort of a region
352    * server and to make sure that the failure path in the region servers is
353    * properly evaluated. But it is difficult to ensure that the region server
354    * doesn't finish the log splitting before it aborts. Also now, there is
355    * this code path where the master will preempt the region server when master
356    * detects that the region server has aborted.
357    * @throws Exception
358    */
359   @Test (timeout=300000)
360   public void testWorkerAbort() throws Exception {
361     LOG.info("testWorkerAbort");
362     startCluster(1);
363     final int NUM_LOG_LINES = 10000;
364     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
365     FileSystem fs = master.getMasterFileSystem().getFileSystem();
366 
367     final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
368     HRegionServer hrs = rsts.get(0).getRegionServer();
369     Path rootdir = FSUtils.getRootDir(conf);
370     final Path logDir = new Path(rootdir,
371         HLog.getHLogDirectoryName(hrs.getServerName().toString()));
372 
373     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
374         "table", "family", 40);
375     makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
376         NUM_LOG_LINES, 100);
377 
378     new Thread() {
379       public void run() {
380         waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
381         for (RegionServerThread rst : rsts) {
382           rst.getRegionServer().abort("testing");
383         }
384       }
385     }.start();
386     // slm.splitLogDistributed(logDir);
387     FileStatus[] logfiles = fs.listStatus(logDir);
388     TaskBatch batch = new TaskBatch();
389     slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
390     //waitForCounter but for one of the 2 counters
391     long curt = System.currentTimeMillis();
392     long waitTime = 80000;
393     long endt = curt + waitTime;
394     while (curt < endt) {
395       if ((tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
396           tot_wkr_final_transistion_failed.get() + tot_wkr_task_done.get() +
397           tot_wkr_preempt_task.get()) == 0) {
398         Thread.yield();
399         curt = System.currentTimeMillis();
400       } else {
401         assertEquals(1, (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
402             tot_wkr_final_transistion_failed.get() + tot_wkr_task_done.get() +
403             tot_wkr_preempt_task.get()));
404         return;
405       }
406     }
407     fail("none of the following counters went up in " + waitTime + 
408         " milliseconds - " +
409         "tot_wkr_task_resigned, tot_wkr_task_err, " +
410         "tot_wkr_final_transistion_failed, tot_wkr_task_done, " +
411         "tot_wkr_preempt_task");
412   }
413 
414   @Test(timeout=30000)
415   public void testDelayedDeleteOnFailure() throws Exception {
416     LOG.info("testDelayedDeleteOnFailure");
417     startCluster(1);
418     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
419     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
420     final Path logDir = new Path(FSUtils.getRootDir(conf), "x");
421     fs.mkdirs(logDir);
422     ExecutorService executor = null;
423     try {
424       final Path corruptedLogFile = new Path(logDir, "x");
425       FSDataOutputStream out;
426       out = fs.create(corruptedLogFile);
427       out.write(0);
428       out.write(Bytes.toBytes("corrupted bytes"));
429       out.close();
430       slm.ignoreZKDeleteForTesting = true;
431       executor = Executors.newSingleThreadExecutor();
432       Runnable runnable = new Runnable() {
433        @Override
434        public void run() {
435           try {
436             // since the logDir is a fake, corrupted one, so the split log worker
437             // will finish it quickly with error, and this call will fail and throw
438             // an IOException.
439             slm.splitLogDistributed(logDir);
440           } catch (IOException ioe) {
441             try {
442               assertTrue(fs.exists(corruptedLogFile));
443               // this call will block waiting for the task to be removed from the
444               // tasks map which is not going to happen since ignoreZKDeleteForTesting
445               // is set to true, until it is interrupted.
446               slm.splitLogDistributed(logDir);
447             } catch (IOException e) {
448               assertTrue(Thread.currentThread().isInterrupted());
449               return;
450             }
451             fail("did not get the expected IOException from the 2nd call");
452           }
453           fail("did not get the expected IOException from the 1st call");
454         }
455       };
456       Future<?> result = executor.submit(runnable);
457       try {
458         result.get(2000, TimeUnit.MILLISECONDS);
459       } catch (TimeoutException te) {
460         // it is ok, expected.
461       }
462       waitForCounter(tot_mgr_wait_for_zk_delete, 0, 1, 10000);
463       executor.shutdownNow();
464       executor = null;
465 
466       // make sure the runnable is finished with no exception thrown.
467       result.get();
468     } finally {
469       if (executor != null) {
470         // interrupt the thread in case the test fails in the middle.
471         // it has no effect if the thread is already terminated.
472         executor.shutdownNow();
473       }
474       fs.delete(logDir, true);
475     }
476   }
477 
478   HTable installTable(ZooKeeperWatcher zkw, String tname, String fname,
479       int nrs ) throws Exception {
480     // Create a table with regions
481     byte [] table = Bytes.toBytes(tname);
482     byte [] family = Bytes.toBytes(fname);
483     LOG.info("Creating table with " + nrs + " regions");
484     HTable ht = TEST_UTIL.createTable(table, family);
485     int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, nrs);
486     assertEquals(nrs, numRegions);
487       LOG.info("Waiting for no more RIT\n");
488     blockUntilNoRIT(zkw, master);
489     // disable-enable cycle to get rid of table's dead regions left behind
490     // by createMultiRegions
491     LOG.debug("Disabling table\n");
492     TEST_UTIL.getHBaseAdmin().disableTable(table);
493     LOG.debug("Waiting for no more RIT\n");
494     blockUntilNoRIT(zkw, master);
495     NavigableSet<String> regions = getAllOnlineRegions(cluster);
496     LOG.debug("Verifying only catalog regions are assigned\n");
497     if (regions.size() != 2) {
498       for (String oregion : regions)
499         LOG.debug("Region still online: " + oregion);
500     }
501     assertEquals(2, regions.size());
502     LOG.debug("Enabling table\n");
503     TEST_UTIL.getHBaseAdmin().enableTable(table);
504     LOG.debug("Waiting for no more RIT\n");
505     blockUntilNoRIT(zkw, master);
506     LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
507     regions = getAllOnlineRegions(cluster);
508     assertEquals(numRegions + 2, regions.size());
509     return ht;
510   }
511 
512   void populateDataInTable(int nrows, String fname) throws Exception {
513     byte [] family = Bytes.toBytes(fname);
514 
515     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
516     assertEquals(NUM_RS, rsts.size());
517 
518     for (RegionServerThread rst : rsts) {
519       HRegionServer hrs = rst.getRegionServer();
520       List<HRegionInfo> hris = hrs.getOnlineRegions();
521       for (HRegionInfo hri : hris) {
522         if (hri.isMetaTable()) {
523           continue;
524         }
525         LOG.debug("adding data to rs = " + rst.getName() +
526             " region = "+ hri.getRegionNameAsString());
527         HRegion region = hrs.getOnlineRegion(hri.getRegionName());
528         assertTrue(region != null);
529         putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
530       }
531     }
532   }
533 
534   public void makeHLog(HLog log,
535       List<HRegionInfo> hris, String tname,
536       int num_edits, int edit_size) throws IOException {
537 
538     // remove root and meta region
539     hris.remove(HRegionInfo.ROOT_REGIONINFO);
540     hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
541     byte[] table = Bytes.toBytes(tname);
542     HTableDescriptor htd = new HTableDescriptor(tname);
543     byte[] value = new byte[edit_size];
544     for (int i = 0; i < edit_size; i++) {
545       value[i] = (byte) ('a' + (i % 26));
546     }
547     int n = hris.size();
548     int[] counts = new int[n];
549     if (n > 0) {
550       for (int i = 0; i < num_edits; i += 1) {
551         WALEdit e = new WALEdit();
552         HRegionInfo curRegionInfo = hris.get(i % n);
553         byte[] startRow = curRegionInfo.getStartKey();
554         if (startRow == null || startRow.length == 0) {
555           startRow = new byte[] { 0, 0, 0, 0, 1 };
556         }
557         byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
558         row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
559                                              // HBaseTestingUtility.createMultiRegions use 5 bytes
560                                              // key
561         byte[] family = Bytes.toBytes("f");
562         byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
563         e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
564         log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
565         counts[i % n] += 1;
566       }
567     }
568     log.sync();
569     log.close();
570     for (int i = 0; i < n; i++) {
571       LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
572     }
573     return;
574   }
575 
576   private int countHLog(Path log, FileSystem fs, Configuration conf)
577   throws IOException {
578     int count = 0;
579     HLog.Reader in = HLog.getReader(fs, log, conf);
580     while (in.next() != null) {
581       count++;
582     }
583     return count;
584   }
585 
586   private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
587   throws KeeperException, InterruptedException {
588     ZKAssign.blockUntilNoRIT(zkw);
589     master.assignmentManager.waitUntilNoRegionsInTransition(60000);
590   }
591 
592   private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
593       byte [] ...families)
594   throws IOException {
595     for(int i = 0; i < numRows; i++) {
596       Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
597       for(byte [] family : families) {
598         put.add(family, qf, null);
599       }
600       region.put(put);
601     }
602   }
603 
604   private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
605       throws IOException {
606     NavigableSet<String> online = new TreeSet<String>();
607     for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
608       for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
609         online.add(region.getRegionNameAsString());
610       }
611     }
612     return online;
613   }
614 
615   private void waitForCounter(AtomicLong ctr, long oldval, long newval,
616       long timems) {
617     long curt = System.currentTimeMillis();
618     long endt = curt + timems;
619     while (curt < endt) {
620       if (ctr.get() == oldval) {
621         Thread.yield();
622         curt = System.currentTimeMillis();
623       } else {
624         assertEquals(newval, ctr.get());
625         return;
626       }
627     }
628     assertTrue(false);
629   }
630 
631   private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
632     for (MasterThread mt : cluster.getLiveMasterThreads()) {
633       if (mt.getMaster().isActiveMaster()) {
634         mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
635         mt.join();
636         break;
637       }
638     }
639     LOG.debug("Master is aborted");
640   }
641 
642   private void startMasterTillNoDeadServers(MiniHBaseCluster cluster)
643       throws IOException, InterruptedException {
644     cluster.startMaster();
645     HMaster master = cluster.getMaster();
646     while (!master.isInitialized()) {
647       Thread.sleep(100);
648     }
649     ServerManager serverManager = master.getServerManager();
650     while (serverManager.areDeadServersInProgress()) {
651       Thread.sleep(100);
652     }
653   }
654 
655   @org.junit.Rule
656   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
657     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
658 }
659