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