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