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