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