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