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 = new HBaseTestingUtility(conf);
136     TEST_UTIL.setDFSCluster(dfsCluster);
137     TEST_UTIL.setZkCluster(zkCluster);
138     TEST_UTIL.startMiniHBaseCluster(num_master, num_rs);
139     cluster = TEST_UTIL.getHBaseCluster();
140     LOG.info("Waiting for active/ready master");
141     cluster.waitForActiveAndReadyMaster();
142     master = cluster.getMaster();
143     while (cluster.getLiveRegionServerThreads().size() < num_rs) {
144       Threads.sleep(1);
145     }
146   }
147 
148   @Before
149   public void before() throws Exception {
150     // refresh configuration
151     conf = HBaseConfiguration.create(originalConf);
152   }
153   
154   @After
155   public void after() throws Exception {
156     if (TEST_UTIL.getHBaseCluster() != null) {
157       for (MasterThread mt : TEST_UTIL.getHBaseCluster().getLiveMasterThreads()) {
158         mt.getMaster().abort("closing...", new Exception("Trace info"));
159       }
160     }
161     TEST_UTIL.shutdownMiniHBaseCluster();
162     TEST_UTIL.getTestFileSystem().delete(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), true);
163     ZKUtil.deleteNodeRecursively(HBaseTestingUtility.getZooKeeperWatcher(TEST_UTIL), "/hbase");
164   }
165 
166   @Test (timeout=300000)
167   public void testThreeRSAbort() throws Exception {
168     LOG.info("testThreeRSAbort");
169     final int NUM_REGIONS_TO_CREATE = 40;
170     final int NUM_ROWS_PER_REGION = 100;
171 
172     startCluster(NUM_RS); // NUM_RS=6.
173 
174     ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf,
175         "distributed log splitting test", null);
176 
177     HTable ht = installTable(zkw, "table", "family", NUM_REGIONS_TO_CREATE);
178     populateDataInTable(NUM_ROWS_PER_REGION, "family");
179 
180 
181     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
182     assertEquals(NUM_RS, rsts.size());
183     rsts.get(0).getRegionServer().abort("testing");
184     rsts.get(1).getRegionServer().abort("testing");
185     rsts.get(2).getRegionServer().abort("testing");
186 
187     long start = EnvironmentEdgeManager.currentTimeMillis();
188     while (cluster.getLiveRegionServerThreads().size() > (NUM_RS - 3)) {
189       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
190         assertTrue(false);
191       }
192       Thread.sleep(200);
193     }
194 
195     start = EnvironmentEdgeManager.currentTimeMillis();
196     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
197       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
198         assertTrue(false);
199       }
200       Thread.sleep(200);
201     }
202 
203     assertEquals(NUM_REGIONS_TO_CREATE * NUM_ROWS_PER_REGION,
204         TEST_UTIL.countRows(ht));
205     ht.close();
206   }
207 
208   @Test (timeout=300000)
209   public void testRecoveredEdits() throws Exception {
210     LOG.info("testRecoveredEdits");
211     startCluster(NUM_RS);
212     final int NUM_LOG_LINES = 1000;
213     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
214     // turn off load balancing to prevent regions from moving around otherwise
215     // they will consume recovered.edits
216     master.balanceSwitch(false);
217     FileSystem fs = master.getMasterFileSystem().getFileSystem();
218 
219     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
220     
221     Path rootdir = FSUtils.getRootDir(conf);
222 
223     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
224         "table", "family", 40);
225     byte[] table = Bytes.toBytes("table");
226     List<HRegionInfo> regions = null;
227     HRegionServer hrs = null;
228     for (int i = 0; i < NUM_RS; i++) {
229       boolean foundRS = false;
230       hrs = rsts.get(i).getRegionServer();
231       regions = hrs.getOnlineRegions();
232       for (HRegionInfo region : regions) {
233         if (region.getTableNameAsString().equalsIgnoreCase("table")) {
234           foundRS = true;
235           break;
236         }
237       }
238       if (foundRS) break;
239     }
240     final Path logDir = new Path(rootdir, HLog.getHLogDirectoryName(hrs
241         .getServerName().toString()));
242     
243     LOG.info("#regions = " + regions.size());
244     Iterator<HRegionInfo> it = regions.iterator();
245     while (it.hasNext()) {
246       HRegionInfo region = it.next();
247       if (region.isMetaTable()) {
248         it.remove();
249       }
250     }
251     makeHLog(hrs.getWAL(), regions, "table",
252         NUM_LOG_LINES, 100);
253 
254     slm.splitLogDistributed(logDir);
255 
256     int count = 0;
257     for (HRegionInfo hri : regions) {
258 
259       Path tdir = HTableDescriptor.getTableDir(rootdir, table);
260       Path editsdir =
261         HLog.getRegionDirRecoveredEditsDir(HRegion.getRegionDir(tdir,
262         hri.getEncodedName()));
263       LOG.debug("checking edits dir " + editsdir);
264       FileStatus[] files = fs.listStatus(editsdir);
265       assertEquals(1, files.length);
266       int c = countHLog(files[0].getPath(), fs, conf);
267       count += c;
268       LOG.info(c + " edits in " + files[0].getPath());
269     }
270     assertEquals(NUM_LOG_LINES, count);
271   }
272 
273   @Test(timeout = 300000)
274   public void testMasterStartsUpWithLogSplittingWork() throws Exception {
275     LOG.info("testMasterStartsUpWithLogSplittingWork");
276     Configuration curConf = HBaseConfiguration.create();
277     curConf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, NUM_RS - 1);
278     startCluster(2, NUM_RS, curConf);
279 
280     final int NUM_REGIONS_TO_CREATE = 40;
281     final int NUM_LOG_LINES = 1000;
282     // turn off load balancing to prevent regions from moving around otherwise
283     // they will consume recovered.edits
284     master.balanceSwitch(false);
285 
286     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
287     final ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "table-creation", null);
288     HTable ht = installTable(zkw, "table", "f", NUM_REGIONS_TO_CREATE);
289 
290     List<HRegionInfo> regions = null;
291     HRegionServer hrs = null;
292     for (int i = 0; i < NUM_RS; i++) {
293       boolean isCarryingMeta = false;
294       hrs = rsts.get(i).getRegionServer();
295       regions = hrs.getOnlineRegions();
296       for (HRegionInfo region : regions) {
297         if (region.isRootRegion() || region.isMetaRegion()) {
298           isCarryingMeta = true;
299           break;
300         }
301       }
302       if (isCarryingMeta) {
303         continue;
304       }
305       break;
306     }
307 
308     LOG.info("#regions = " + regions.size());
309     Iterator<HRegionInfo> it = regions.iterator();
310     while (it.hasNext()) {
311       HRegionInfo region = it.next();
312       if (region.isMetaTable()) {
313         it.remove();
314       }
315     }
316     makeHLog(hrs.getWAL(), regions, "table", NUM_LOG_LINES, 100);
317 
318     // abort master
319     abortMaster(cluster);
320 
321     // abort RS
322     int numRS = cluster.getLiveRegionServerThreads().size();
323     LOG.info("Aborting region server: " + hrs.getServerName());
324     hrs.abort("testing");
325 
326     // wait for the RS dies
327     long start = EnvironmentEdgeManager.currentTimeMillis();
328     while (cluster.getLiveRegionServerThreads().size() > (numRS - 1)) {
329       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
330         assertTrue(false);
331       }
332       Thread.sleep(200);
333     }
334 
335     Thread.sleep(2000);
336     LOG.info("Current Open Regions:" + getAllOnlineRegions(cluster).size());
337     
338     startMasterTillNoDeadServers(cluster);
339     
340     start = EnvironmentEdgeManager.currentTimeMillis();
341     while (getAllOnlineRegions(cluster).size() < (NUM_REGIONS_TO_CREATE + 2)) {
342       if (EnvironmentEdgeManager.currentTimeMillis() - start > 60000) {
343         assertTrue("Timedout", false);
344       }
345       Thread.sleep(200);
346     }
347 
348     LOG.info("Current Open Regions After Master Node Starts Up:"
349         + getAllOnlineRegions(cluster).size());
350 
351     assertEquals(NUM_LOG_LINES, TEST_UTIL.countRows(ht));
352 
353     ht.close();
354   }
355 
356   /**
357    * The original intention of this test was to force an abort of a region
358    * server and to make sure that the failure path in the region servers is
359    * properly evaluated. But it is difficult to ensure that the region server
360    * doesn't finish the log splitting before it aborts. Also now, there is
361    * this code path where the master will preempt the region server when master
362    * detects that the region server has aborted.
363    * @throws Exception
364    */
365   @Test (timeout=300000)
366   public void testWorkerAbort() throws Exception {
367     LOG.info("testWorkerAbort");
368     startCluster(1);
369     final int NUM_LOG_LINES = 10000;
370     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
371     FileSystem fs = master.getMasterFileSystem().getFileSystem();
372 
373     final List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
374     HRegionServer hrs = rsts.get(0).getRegionServer();
375     Path rootdir = FSUtils.getRootDir(conf);
376     final Path logDir = new Path(rootdir,
377         HLog.getHLogDirectoryName(hrs.getServerName().toString()));
378 
379     installTable(new ZooKeeperWatcher(conf, "table-creation", null),
380         "table", "family", 40);
381     makeHLog(hrs.getWAL(), hrs.getOnlineRegions(), "table",
382         NUM_LOG_LINES, 100);
383 
384     new Thread() {
385       public void run() {
386         waitForCounter(tot_wkr_task_acquired, 0, 1, 1000);
387         for (RegionServerThread rst : rsts) {
388           rst.getRegionServer().abort("testing");
389         }
390       }
391     }.start();
392     // slm.splitLogDistributed(logDir);
393     FileStatus[] logfiles = fs.listStatus(logDir);
394     TaskBatch batch = new TaskBatch();
395     slm.enqueueSplitTask(logfiles[0].getPath().toString(), batch);
396     //waitForCounter but for one of the 2 counters
397     long curt = System.currentTimeMillis();
398     long waitTime = 80000;
399     long endt = curt + waitTime;
400     while (curt < endt) {
401       if ((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()) == 0) {
404         Thread.yield();
405         curt = System.currentTimeMillis();
406       } else {
407         assertEquals(1, (tot_wkr_task_resigned.get() + tot_wkr_task_err.get() + 
408             tot_wkr_final_transistion_failed.get() + tot_wkr_task_done.get() +
409             tot_wkr_preempt_task.get()));
410         return;
411       }
412     }
413     fail("none of the following counters went up in " + waitTime + 
414         " milliseconds - " +
415         "tot_wkr_task_resigned, tot_wkr_task_err, " +
416         "tot_wkr_final_transistion_failed, tot_wkr_task_done, " +
417         "tot_wkr_preempt_task");
418   }
419 
420   @Test(timeout=30000)
421   public void testDelayedDeleteOnFailure() throws Exception {
422     LOG.info("testDelayedDeleteOnFailure");
423     startCluster(1);
424     final SplitLogManager slm = master.getMasterFileSystem().splitLogManager;
425     final FileSystem fs = master.getMasterFileSystem().getFileSystem();
426     final Path logDir = new Path(FSUtils.getRootDir(conf), "x");
427     fs.mkdirs(logDir);
428     ExecutorService executor = null;
429     try {
430       final Path corruptedLogFile = new Path(logDir, "x");
431       FSDataOutputStream out;
432       out = fs.create(corruptedLogFile);
433       out.write(0);
434       out.write(Bytes.toBytes("corrupted bytes"));
435       out.close();
436       slm.ignoreZKDeleteForTesting = true;
437       executor = Executors.newSingleThreadExecutor();
438       Runnable runnable = new Runnable() {
439        @Override
440        public void run() {
441           try {
442             // since the logDir is a fake, corrupted one, so the split log worker
443             // will finish it quickly with error, and this call will fail and throw
444             // an IOException.
445             slm.splitLogDistributed(logDir);
446           } catch (IOException ioe) {
447             try {
448               assertTrue(fs.exists(corruptedLogFile));
449               // this call will block waiting for the task to be removed from the
450               // tasks map which is not going to happen since ignoreZKDeleteForTesting
451               // is set to true, until it is interrupted.
452               slm.splitLogDistributed(logDir);
453             } catch (IOException e) {
454               assertTrue(Thread.currentThread().isInterrupted());
455               return;
456             }
457             fail("did not get the expected IOException from the 2nd call");
458           }
459           fail("did not get the expected IOException from the 1st call");
460         }
461       };
462       Future<?> result = executor.submit(runnable);
463       try {
464         result.get(2000, TimeUnit.MILLISECONDS);
465       } catch (TimeoutException te) {
466         // it is ok, expected.
467       }
468       waitForCounter(tot_mgr_wait_for_zk_delete, 0, 1, 10000);
469       executor.shutdownNow();
470       executor = null;
471 
472       // make sure the runnable is finished with no exception thrown.
473       result.get();
474     } finally {
475       if (executor != null) {
476         // interrupt the thread in case the test fails in the middle.
477         // it has no effect if the thread is already terminated.
478         executor.shutdownNow();
479       }
480       fs.delete(logDir, true);
481     }
482   }
483 
484   HTable installTable(ZooKeeperWatcher zkw, String tname, String fname,
485       int nrs ) throws Exception {
486     // Create a table with regions
487     byte [] table = Bytes.toBytes(tname);
488     byte [] family = Bytes.toBytes(fname);
489     LOG.info("Creating table with " + nrs + " regions");
490     HTable ht = TEST_UTIL.createTable(table, family);
491     int numRegions = TEST_UTIL.createMultiRegions(conf, ht, family, nrs);
492     assertEquals(nrs, numRegions);
493       LOG.info("Waiting for no more RIT\n");
494     blockUntilNoRIT(zkw, master);
495     // disable-enable cycle to get rid of table's dead regions left behind
496     // by createMultiRegions
497     LOG.debug("Disabling table\n");
498     TEST_UTIL.getHBaseAdmin().disableTable(table);
499     LOG.debug("Waiting for no more RIT\n");
500     blockUntilNoRIT(zkw, master);
501     NavigableSet<String> regions = getAllOnlineRegions(cluster);
502     LOG.debug("Verifying only catalog regions are assigned\n");
503     if (regions.size() != 2) {
504       for (String oregion : regions)
505         LOG.debug("Region still online: " + oregion);
506     }
507     assertEquals(2, regions.size());
508     LOG.debug("Enabling table\n");
509     TEST_UTIL.getHBaseAdmin().enableTable(table);
510     LOG.debug("Waiting for no more RIT\n");
511     blockUntilNoRIT(zkw, master);
512     LOG.debug("Verifying there are " + numRegions + " assigned on cluster\n");
513     regions = getAllOnlineRegions(cluster);
514     assertEquals(numRegions + 2, regions.size());
515     return ht;
516   }
517 
518   void populateDataInTable(int nrows, String fname) throws Exception {
519     byte [] family = Bytes.toBytes(fname);
520 
521     List<RegionServerThread> rsts = cluster.getLiveRegionServerThreads();
522     assertEquals(NUM_RS, rsts.size());
523 
524     for (RegionServerThread rst : rsts) {
525       HRegionServer hrs = rst.getRegionServer();
526       List<HRegionInfo> hris = hrs.getOnlineRegions();
527       for (HRegionInfo hri : hris) {
528         if (hri.isMetaTable()) {
529           continue;
530         }
531         LOG.debug("adding data to rs = " + rst.getName() +
532             " region = "+ hri.getRegionNameAsString());
533         HRegion region = hrs.getOnlineRegion(hri.getRegionName());
534         assertTrue(region != null);
535         putData(region, hri.getStartKey(), nrows, Bytes.toBytes("q"), family);
536       }
537     }
538   }
539 
540   public void makeHLog(HLog log,
541       List<HRegionInfo> hris, String tname,
542       int num_edits, int edit_size) throws IOException {
543 
544     // remove root and meta region
545     hris.remove(HRegionInfo.ROOT_REGIONINFO);
546     hris.remove(HRegionInfo.FIRST_META_REGIONINFO);
547     byte[] table = Bytes.toBytes(tname);
548     HTableDescriptor htd = new HTableDescriptor(tname);
549     byte[] value = new byte[edit_size];
550     for (int i = 0; i < edit_size; i++) {
551       value[i] = (byte) ('a' + (i % 26));
552     }
553     int n = hris.size();
554     int[] counts = new int[n];
555     if (n > 0) {
556       for (int i = 0; i < num_edits; i += 1) {
557         WALEdit e = new WALEdit();
558         HRegionInfo curRegionInfo = hris.get(i % n);
559         byte[] startRow = curRegionInfo.getStartKey();
560         if (startRow == null || startRow.length == 0) {
561           startRow = new byte[] { 0, 0, 0, 0, 1 };
562         }
563         byte[] row = Bytes.incrementBytes(startRow, counts[i % n]);
564         row = Arrays.copyOfRange(row, 3, 8); // use last 5 bytes because
565                                              // HBaseTestingUtility.createMultiRegions use 5 bytes
566                                              // key
567         byte[] family = Bytes.toBytes("f");
568         byte[] qualifier = Bytes.toBytes("c" + Integer.toString(i));
569         e.add(new KeyValue(row, family, qualifier, System.currentTimeMillis(), value));
570         log.append(curRegionInfo, table, e, System.currentTimeMillis(), htd);
571         counts[i % n] += 1;
572       }
573     }
574     log.sync();
575     log.close();
576     for (int i = 0; i < n; i++) {
577       LOG.info("region " + hris.get(i).getRegionNameAsString() + " has " + counts[i] + " edits");
578     }
579     return;
580   }
581 
582   private int countHLog(Path log, FileSystem fs, Configuration conf)
583   throws IOException {
584     int count = 0;
585     HLog.Reader in = HLog.getReader(fs, log, conf);
586     while (in.next() != null) {
587       count++;
588     }
589     return count;
590   }
591 
592   private void blockUntilNoRIT(ZooKeeperWatcher zkw, HMaster master)
593   throws KeeperException, InterruptedException {
594     ZKAssign.blockUntilNoRIT(zkw);
595     master.assignmentManager.waitUntilNoRegionsInTransition(60000);
596   }
597 
598   private void putData(HRegion region, byte[] startRow, int numRows, byte [] qf,
599       byte [] ...families)
600   throws IOException {
601     for(int i = 0; i < numRows; i++) {
602       Put put = new Put(Bytes.add(startRow, Bytes.toBytes(i)));
603       for(byte [] family : families) {
604         put.add(family, qf, null);
605       }
606       region.put(put);
607     }
608   }
609 
610   private NavigableSet<String> getAllOnlineRegions(MiniHBaseCluster cluster)
611       throws IOException {
612     NavigableSet<String> online = new TreeSet<String>();
613     for (RegionServerThread rst : cluster.getLiveRegionServerThreads()) {
614       for (HRegionInfo region : rst.getRegionServer().getOnlineRegions()) {
615         online.add(region.getRegionNameAsString());
616       }
617     }
618     return online;
619   }
620 
621   private void waitForCounter(AtomicLong ctr, long oldval, long newval,
622       long timems) {
623     long curt = System.currentTimeMillis();
624     long endt = curt + timems;
625     while (curt < endt) {
626       if (ctr.get() == oldval) {
627         Thread.yield();
628         curt = System.currentTimeMillis();
629       } else {
630         assertEquals(newval, ctr.get());
631         return;
632       }
633     }
634     assertTrue(false);
635   }
636 
637   private void abortMaster(MiniHBaseCluster cluster) throws InterruptedException {
638     for (MasterThread mt : cluster.getLiveMasterThreads()) {
639       if (mt.getMaster().isActiveMaster()) {
640         mt.getMaster().abort("Aborting for tests", new Exception("Trace info"));
641         mt.join();
642         break;
643       }
644     }
645     LOG.debug("Master is aborted");
646   }
647 
648   private void startMasterTillNoDeadServers(MiniHBaseCluster cluster)
649       throws IOException, InterruptedException {
650     cluster.startMaster();
651     HMaster master = cluster.getMaster();
652     while (!master.isInitialized()) {
653       Thread.sleep(100);
654     }
655     ServerManager serverManager = master.getServerManager();
656     while (serverManager.areDeadServersInProgress()) {
657       Thread.sleep(100);
658     }
659   }
660 
661   @org.junit.Rule
662   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
663     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
664 }
665