1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.master;
20
21 import static org.apache.hadoop.hbase.SplitLogCounters.resetCounters;
22 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_get_data_nonode;
23 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_heartbeat;
24 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_log_split_batch_success;
25 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_queued;
26 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_node_create_result;
27 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_orphan_task_acquired;
28 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan;
29 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_rescan_deleted;
30 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit;
31 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_dead_server_task;
32 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_failed;
33 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_force;
34 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_threshold_reached;
35 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_resubmit_unassigned;
36 import static org.apache.hadoop.hbase.SplitLogCounters.tot_mgr_task_deleted;
37 import static org.junit.Assert.assertEquals;
38 import static org.junit.Assert.assertFalse;
39 import static org.junit.Assert.assertTrue;
40
41 import java.io.IOException;
42 import java.util.List;
43 import java.util.UUID;
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.FileSystem;
50 import org.apache.hadoop.fs.Path;
51 import org.apache.hadoop.hbase.HBaseTestingUtility;
52 import org.apache.hadoop.hbase.HConstants;
53 import org.apache.hadoop.hbase.HRegionInfo;
54 import org.apache.hadoop.hbase.MediumTests;
55 import org.apache.hadoop.hbase.ServerName;
56 import org.apache.hadoop.hbase.SplitLogTask;
57 import org.apache.hadoop.hbase.Stoppable;
58 import org.apache.hadoop.hbase.Waiter;
59 import org.apache.hadoop.hbase.master.SplitLogManager.Task;
60 import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
61 import org.apache.hadoop.hbase.regionserver.TestMasterAddressTracker.NodeCreationListener;
62 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
63 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
64 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
65 import org.apache.log4j.Level;
66 import org.apache.log4j.Logger;
67 import org.apache.zookeeper.CreateMode;
68 import org.apache.zookeeper.KeeperException;
69 import org.apache.zookeeper.ZooDefs.Ids;
70 import org.junit.After;
71 import org.junit.Assert;
72 import org.junit.Before;
73 import org.junit.Test;
74 import org.junit.experimental.categories.Category;
75 import org.mockito.Mockito;
76
77 @Category(MediumTests.class)
78 public class TestSplitLogManager {
79 private static final Log LOG = LogFactory.getLog(TestSplitLogManager.class);
80 private final ServerName DUMMY_MASTER = ServerName.valueOf("dummy-master,1,1");
81 private final ServerManager sm = Mockito.mock(ServerManager.class);
82 private final MasterServices master = Mockito.mock(MasterServices.class);
83
84 static {
85 Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
86 }
87
88 private ZooKeeperWatcher zkw;
89 private static boolean stopped = false;
90 private SplitLogManager slm;
91 private Configuration conf;
92 private int to;
93
94 private static HBaseTestingUtility TEST_UTIL;
95
96 static Stoppable stopper = new Stoppable() {
97 @Override
98 public void stop(String why) {
99 stopped = true;
100 }
101
102 @Override
103 public boolean isStopped() {
104 return stopped;
105 }
106
107 };
108
109 @Before
110 public void setup() throws Exception {
111 TEST_UTIL = new HBaseTestingUtility();
112 TEST_UTIL.startMiniZKCluster();
113 conf = TEST_UTIL.getConfiguration();
114
115 zkw = new ZooKeeperWatcher(conf, "split-log-manager-tests" + UUID.randomUUID().toString(), null);
116 ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
117 ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
118 assertTrue(ZKUtil.checkExists(zkw, zkw.baseZNode) != -1);
119 LOG.debug(zkw.baseZNode + " created");
120 ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
121 assertTrue(ZKUtil.checkExists(zkw, zkw.splitLogZNode) != -1);
122 LOG.debug(zkw.splitLogZNode + " created");
123
124 stopped = false;
125 resetCounters();
126
127
128
129 Mockito.when(sm.isServerOnline(Mockito.any(ServerName.class))).thenReturn(true);
130 Mockito.when(master.getServerManager()).thenReturn(sm);
131
132 to = 6000;
133 conf.setInt("hbase.splitlog.manager.timeout", to);
134 conf.setInt("hbase.splitlog.manager.unassigned.timeout", 2 * to);
135 conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 100);
136 to = to + 4 * 100;
137 }
138
139 @After
140 public void teardown() throws IOException, KeeperException {
141 stopper.stop("");
142 slm.stop();
143 TEST_UTIL.shutdownMiniZKCluster();
144 }
145
146 private interface Expr {
147 long eval();
148 }
149
150 private void waitForCounter(final AtomicLong ctr, long oldval, long newval, long timems)
151 throws Exception {
152 Expr e = new Expr() {
153 public long eval() {
154 return ctr.get();
155 }
156 };
157 waitForCounter(e, oldval, newval, timems);
158 return;
159 }
160
161 private void waitForCounter(final Expr e, final long oldval, long newval, long timems)
162 throws Exception {
163
164 TEST_UTIL.waitFor(timems, 10, new Waiter.Predicate<Exception>() {
165 @Override
166 public boolean evaluate() throws Exception {
167 return (e.eval() != oldval);
168 }
169 });
170
171 assertEquals(newval, e.eval());
172 }
173
174 private String submitTaskAndWait(TaskBatch batch, String name)
175 throws KeeperException, InterruptedException {
176 String tasknode = ZKSplitLog.getEncodedNodeName(zkw, name);
177 NodeCreationListener listener = new NodeCreationListener(zkw, tasknode);
178 zkw.registerListener(listener);
179 ZKUtil.watchAndCheckExists(zkw, tasknode);
180
181 slm.enqueueSplitTask(name, batch);
182 assertEquals(1, batch.installed);
183 assertTrue(slm.findOrCreateOrphanTask(tasknode).batch == batch);
184 assertEquals(1L, tot_mgr_node_create_queued.get());
185
186 LOG.debug("waiting for task node creation");
187 listener.waitForCreation();
188 LOG.debug("task created");
189 return tasknode;
190 }
191
192
193
194
195
196 @Test
197 public void testTaskCreation() throws Exception {
198
199 LOG.info("TestTaskCreation - test the creation of a task in zk");
200 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
201 TaskBatch batch = new TaskBatch();
202
203 String tasknode = submitTaskAndWait(batch, "foo/1");
204
205 byte[] data = ZKUtil.getData(zkw, tasknode);
206 SplitLogTask slt = SplitLogTask.parseFrom(data);
207 LOG.info("Task node created " + slt.toString());
208 assertTrue(slt.isUnassigned(DUMMY_MASTER));
209 }
210
211 @Test
212 public void testOrphanTaskAcquisition() throws Exception {
213 LOG.info("TestOrphanTaskAcquisition");
214
215 String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
216 SplitLogTask slt = new SplitLogTask.Owned(DUMMY_MASTER);
217 zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
218 CreateMode.PERSISTENT);
219
220 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
221 waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
222 Task task = slm.findOrCreateOrphanTask(tasknode);
223 assertTrue(task.isOrphan());
224 waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
225 assertFalse(task.isUnassigned());
226 long curt = System.currentTimeMillis();
227 assertTrue((task.last_update <= curt) &&
228 (task.last_update > (curt - 1000)));
229 LOG.info("waiting for manager to resubmit the orphan task");
230 waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
231 assertTrue(task.isUnassigned());
232 waitForCounter(tot_mgr_rescan, 0, 1, to + to/2);
233 }
234
235 @Test
236 public void testUnassignedOrphan() throws Exception {
237 LOG.info("TestUnassignedOrphan - an unassigned task is resubmitted at" +
238 " startup");
239 String tasknode = ZKSplitLog.getEncodedNodeName(zkw, "orphan/test/slash");
240
241 SplitLogTask slt = new SplitLogTask.Unassigned(DUMMY_MASTER);
242 zkw.getRecoverableZooKeeper().create(tasknode, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
243 CreateMode.PERSISTENT);
244 int version = ZKUtil.checkExists(zkw, tasknode);
245
246 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
247 waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
248 Task task = slm.findOrCreateOrphanTask(tasknode);
249 assertTrue(task.isOrphan());
250 assertTrue(task.isUnassigned());
251
252 waitForCounter(tot_mgr_rescan, 0, 1, to/2);
253 Task task2 = slm.findOrCreateOrphanTask(tasknode);
254 assertTrue(task == task2);
255 LOG.debug("task = " + task);
256 assertEquals(1L, tot_mgr_resubmit.get());
257 assertEquals(1, task.incarnation);
258 assertEquals(0, task.unforcedResubmits);
259 assertTrue(task.isOrphan());
260 assertTrue(task.isUnassigned());
261 assertTrue(ZKUtil.checkExists(zkw, tasknode) > version);
262 }
263
264 @Test
265 public void testMultipleResubmits() throws Exception {
266 LOG.info("TestMultipleResbmits - no indefinite resubmissions");
267
268 conf.setInt("hbase.splitlog.max.resubmit", 2);
269 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
270 TaskBatch batch = new TaskBatch();
271
272 String tasknode = submitTaskAndWait(batch, "foo/1");
273 int version = ZKUtil.checkExists(zkw, tasknode);
274 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
275 final ServerName worker2 = ServerName.valueOf("worker2,1,1");
276 final ServerName worker3 = ServerName.valueOf("worker3,1,1");
277 SplitLogTask slt = new SplitLogTask.Owned(worker1);
278 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
279 waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
280 waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
281 int version1 = ZKUtil.checkExists(zkw, tasknode);
282 assertTrue(version1 > version);
283 slt = new SplitLogTask.Owned(worker2);
284 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
285 waitForCounter(tot_mgr_heartbeat, 1, 2, to/2);
286 waitForCounter(tot_mgr_resubmit, 1, 2, to + to/2);
287 int version2 = ZKUtil.checkExists(zkw, tasknode);
288 assertTrue(version2 > version1);
289 slt = new SplitLogTask.Owned(worker3);
290 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
291 waitForCounter(tot_mgr_heartbeat, 2, 3, to/2);
292 waitForCounter(tot_mgr_resubmit_threshold_reached, 0, 1, to + to/2);
293 Thread.sleep(to + to/2);
294 assertEquals(2L, tot_mgr_resubmit.get() - tot_mgr_resubmit_force.get());
295 }
296
297 @Test
298 public void testRescanCleanup() throws Exception {
299 LOG.info("TestRescanCleanup - ensure RESCAN nodes are cleaned up");
300
301 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
302 TaskBatch batch = new TaskBatch();
303
304 String tasknode = submitTaskAndWait(batch, "foo/1");
305 int version = ZKUtil.checkExists(zkw, tasknode);
306 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
307 SplitLogTask slt = new SplitLogTask.Owned(worker1);
308 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
309 waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
310 waitForCounter(new Expr() {
311 @Override
312 public long eval() {
313 return (tot_mgr_resubmit.get() + tot_mgr_resubmit_failed.get());
314 }
315 }, 0, 1, 5*60000);
316 Assert.assertEquals("Could not run test. Lost ZK connection?", 0, tot_mgr_resubmit_failed.get());
317 int version1 = ZKUtil.checkExists(zkw, tasknode);
318 assertTrue(version1 > version);
319 byte[] taskstate = ZKUtil.getData(zkw, tasknode);
320 slt = SplitLogTask.parseFrom(taskstate);
321 assertTrue(slt.isUnassigned(DUMMY_MASTER));
322
323 waitForCounter(tot_mgr_rescan_deleted, 0, 1, to/2);
324 }
325
326 @Test
327 public void testTaskDone() throws Exception {
328 LOG.info("TestTaskDone - cleanup task node once in DONE state");
329
330 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
331 TaskBatch batch = new TaskBatch();
332 String tasknode = submitTaskAndWait(batch, "foo/1");
333 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
334 SplitLogTask slt = new SplitLogTask.Done(worker1);
335 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
336 synchronized (batch) {
337 while (batch.installed != batch.done) {
338 batch.wait();
339 }
340 }
341 waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
342 assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
343 }
344
345 @Test
346 public void testTaskErr() throws Exception {
347 LOG.info("TestTaskErr - cleanup task node once in ERR state");
348
349 conf.setInt("hbase.splitlog.max.resubmit", 0);
350 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
351 TaskBatch batch = new TaskBatch();
352
353 String tasknode = submitTaskAndWait(batch, "foo/1");
354 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
355 SplitLogTask slt = new SplitLogTask.Err(worker1);
356 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
357
358 synchronized (batch) {
359 while (batch.installed != batch.error) {
360 batch.wait();
361 }
362 }
363 waitForCounter(tot_mgr_task_deleted, 0, 1, to/2);
364 assertTrue(ZKUtil.checkExists(zkw, tasknode) == -1);
365 conf.setInt("hbase.splitlog.max.resubmit", SplitLogManager.DEFAULT_MAX_RESUBMIT);
366 }
367
368 @Test
369 public void testTaskResigned() throws Exception {
370 LOG.info("TestTaskResigned - resubmit task node once in RESIGNED state");
371 assertEquals(tot_mgr_resubmit.get(), 0);
372 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
373 assertEquals(tot_mgr_resubmit.get(), 0);
374 TaskBatch batch = new TaskBatch();
375 String tasknode = submitTaskAndWait(batch, "foo/1");
376 assertEquals(tot_mgr_resubmit.get(), 0);
377 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
378 assertEquals(tot_mgr_resubmit.get(), 0);
379 SplitLogTask slt = new SplitLogTask.Resigned(worker1);
380 assertEquals(tot_mgr_resubmit.get(), 0);
381 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
382 int version = ZKUtil.checkExists(zkw, tasknode);
383
384 if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to/2);
385 assertEquals(tot_mgr_resubmit.get(), 1);
386 int version1 = ZKUtil.checkExists(zkw, tasknode);
387 assertTrue("version1=" + version1 + ", version=" + version, version1 > version);
388
389 byte[] taskstate = ZKUtil.getData(zkw, tasknode);
390 slt = SplitLogTask.parseFrom(taskstate);
391 assertTrue(slt.isUnassigned(DUMMY_MASTER));
392 }
393
394 @Test
395 public void testUnassignedTimeout() throws Exception {
396 LOG.info("TestUnassignedTimeout - iff all tasks are unassigned then" +
397 " resubmit");
398
399
400 String tasknode1 = ZKSplitLog.getEncodedNodeName(zkw, "orphan/1");
401 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
402 SplitLogTask slt = new SplitLogTask.Owned(worker1);
403 zkw.getRecoverableZooKeeper().create(tasknode1, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
404 CreateMode.PERSISTENT);
405
406 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
407 waitForCounter(tot_mgr_orphan_task_acquired, 0, 1, to/2);
408
409
410 TaskBatch batch = new TaskBatch();
411 submitTaskAndWait(batch, "foo/1");
412
413
414 for (int i = 0; i < (3 * to)/100; i++) {
415 Thread.sleep(100);
416 final ServerName worker2 = ServerName.valueOf("worker1,1,1");
417 slt = new SplitLogTask.Owned(worker2);
418 ZKUtil.setData(zkw, tasknode1, slt.toByteArray());
419 }
420
421
422
423 LOG.info("waiting for manager to resubmit the orphan task");
424 waitForCounter(tot_mgr_resubmit, 0, 1, to + to/2);
425
426
427 waitForCounter(tot_mgr_resubmit_unassigned, 0, 1, 2 * to + to/2);
428 }
429
430 @Test
431 public void testDeadWorker() throws Exception {
432 LOG.info("testDeadWorker");
433
434 conf.setLong("hbase.splitlog.max.resubmit", 0);
435 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
436 TaskBatch batch = new TaskBatch();
437
438 String tasknode = submitTaskAndWait(batch, "foo/1");
439 int version = ZKUtil.checkExists(zkw, tasknode);
440 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
441 SplitLogTask slt = new SplitLogTask.Owned(worker1);
442 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
443 if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
444 slm.handleDeadWorker(worker1);
445 if (tot_mgr_resubmit.get() == 0) waitForCounter(tot_mgr_resubmit, 0, 1, to+to/2);
446 if (tot_mgr_resubmit_dead_server_task.get() == 0) {
447 waitForCounter(tot_mgr_resubmit_dead_server_task, 0, 1, to + to/2);
448 }
449
450 int version1 = ZKUtil.checkExists(zkw, tasknode);
451 assertTrue(version1 > version);
452 byte[] taskstate = ZKUtil.getData(zkw, tasknode);
453 slt = SplitLogTask.parseFrom(taskstate);
454 assertTrue(slt.isUnassigned(DUMMY_MASTER));
455 return;
456 }
457
458 @Test
459 public void testWorkerCrash() throws Exception {
460 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
461 TaskBatch batch = new TaskBatch();
462
463 String tasknode = submitTaskAndWait(batch, "foo/1");
464 final ServerName worker1 = ServerName.valueOf("worker1,1,1");
465
466 SplitLogTask slt = new SplitLogTask.Owned(worker1);
467 ZKUtil.setData(zkw, tasknode, slt.toByteArray());
468 if (tot_mgr_heartbeat.get() == 0) waitForCounter(tot_mgr_heartbeat, 0, 1, to/2);
469
470
471 Assert.assertEquals(0, tot_mgr_resubmit.get());
472
473
474 Mockito.when(sm.isServerOnline(worker1)).thenReturn(false);
475
476 Thread.sleep(1300);
477
478
479 Assert.assertEquals(1, tot_mgr_resubmit.get());
480 }
481
482 @Test
483 public void testEmptyLogDir() throws Exception {
484 LOG.info("testEmptyLogDir");
485 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
486 FileSystem fs = TEST_UTIL.getTestFileSystem();
487 Path emptyLogDirPath = new Path(fs.getWorkingDirectory(),
488 UUID.randomUUID().toString());
489 fs.mkdirs(emptyLogDirPath);
490 slm.splitLogDistributed(emptyLogDirPath);
491 assertFalse(fs.exists(emptyLogDirPath));
492 }
493
494 @Test(timeout=45000)
495 public void testVanishingTaskZNode() throws Exception {
496 LOG.info("testVanishingTaskZNode");
497
498 conf.setInt("hbase.splitlog.manager.unassigned.timeout", 0);
499 conf.setInt("hbase.splitlog.manager.timeoutmonitor.period", 1000);
500 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
501 FileSystem fs = TEST_UTIL.getTestFileSystem();
502 final Path logDir = new Path(fs.getWorkingDirectory(),
503 UUID.randomUUID().toString());
504 fs.mkdirs(logDir);
505 Thread thread = null;
506 try {
507 Path logFile = new Path(logDir, UUID.randomUUID().toString());
508 fs.createNewFile(logFile);
509 thread = new Thread() {
510 public void run() {
511 try {
512
513
514
515 slm.splitLogDistributed(logDir);
516 } catch (Exception e) {
517 LOG.warn("splitLogDistributed failed", e);
518 }
519 }
520 };
521 thread.start();
522 waitForCounter(tot_mgr_node_create_result, 0, 1, 10000);
523 String znode = ZKSplitLog.getEncodedNodeName(zkw, logFile.toString());
524
525 ZKUtil.deleteNode(zkw, znode);
526 waitForCounter(tot_mgr_get_data_nonode, 0, 1, 30000);
527 waitForCounter(tot_mgr_log_split_batch_success, 0, 1, to/2);
528 assertTrue(fs.exists(logFile));
529 } finally {
530 if (thread != null) {
531
532
533 thread.interrupt();
534 }
535 fs.delete(logDir, true);
536 }
537 }
538
539
540
541
542
543
544 @Test(timeout = 300000)
545 public void testRecoveryRegionRemovedFromZK() throws Exception {
546 LOG.info("testRecoveryRegionRemovedFromZK");
547 conf.setBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false);
548 String nodePath =
549 ZKUtil.joinZNode(zkw.recoveringRegionsZNode,
550 HRegionInfo.FIRST_META_REGIONINFO.getEncodedName());
551 ZKUtil.createSetData(zkw, nodePath, ZKUtil.positionToByteArray(0L));
552
553 slm = new SplitLogManager(zkw, conf, stopper, master, DUMMY_MASTER);
554 slm.removeStaleRecoveringRegionsFromZK(null);
555
556 List<String> recoveringRegions =
557 zkw.getRecoverableZooKeeper().getChildren(zkw.recoveringRegionsZNode, false);
558
559 assertTrue("Recovery regions isn't cleaned", recoveringRegions.isEmpty());
560 }
561 }