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