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