1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import static org.hamcrest.CoreMatchers.is;
22 import static org.hamcrest.CoreMatchers.not;
23 import static org.junit.Assert.assertEquals;
24 import static org.junit.Assert.assertThat;
25 import static org.junit.Assert.assertTrue;
26 import static org.mockito.Mockito.mock;
27 import static org.mockito.Mockito.when;
28
29 import java.util.List;
30 import java.util.concurrent.atomic.AtomicLong;
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.hbase.CoordinatedStateManager;
36 import org.apache.hadoop.hbase.CoordinatedStateManagerFactory;
37 import org.apache.hadoop.hbase.HBaseConfiguration;
38 import org.apache.hadoop.hbase.HBaseTestingUtility;
39 import org.apache.hadoop.hbase.HConstants;
40 import org.apache.hadoop.hbase.testclassification.MediumTests;
41 import org.apache.hadoop.hbase.Server;
42 import org.apache.hadoop.hbase.ServerName;
43 import org.apache.hadoop.hbase.SplitLogCounters;
44 import org.apache.hadoop.hbase.SplitLogTask;
45 import org.apache.hadoop.hbase.Waiter;
46 import org.apache.hadoop.hbase.client.ClusterConnection;
47 import org.apache.hadoop.hbase.executor.ExecutorService;
48 import org.apache.hadoop.hbase.executor.ExecutorType;
49 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
50 import org.apache.hadoop.hbase.util.CancelableProgressable;
51 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
52 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
53 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
54 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
55 import org.apache.log4j.Level;
56 import org.apache.log4j.Logger;
57 import org.apache.zookeeper.CreateMode;
58 import org.apache.zookeeper.ZooDefs.Ids;
59 import org.junit.After;
60 import org.junit.Before;
61 import org.junit.Test;
62 import org.junit.experimental.categories.Category;
63
64 @Category(MediumTests.class)
65 public class TestSplitLogWorker {
66 private static final Log LOG = LogFactory.getLog(TestSplitLogWorker.class);
67 private static final int WAIT_TIME = 15000;
68 private final ServerName MANAGER = ServerName.valueOf("manager,1,1");
69 static {
70 Logger.getLogger("org.apache.hadoop.hbase").setLevel(Level.DEBUG);
71 }
72 private final static HBaseTestingUtility TEST_UTIL =
73 new HBaseTestingUtility();
74 private DummyServer ds;
75 private ZooKeeperWatcher zkw;
76 private SplitLogWorker slw;
77 private ExecutorService executorService;
78 private RecoveryMode mode;
79
80 class DummyServer implements Server {
81 private ZooKeeperWatcher zkw;
82 private Configuration conf;
83 private CoordinatedStateManager cm;
84
85 public DummyServer(ZooKeeperWatcher zkw, Configuration conf) {
86 this.zkw = zkw;
87 this.conf = conf;
88 cm = CoordinatedStateManagerFactory.getCoordinatedStateManager(conf);
89 cm.initialize(this);
90 }
91
92 @Override
93 public void abort(String why, Throwable e) {
94 }
95
96 @Override
97 public boolean isAborted() {
98 return false;
99 }
100
101 @Override
102 public void stop(String why) {
103 }
104
105 @Override
106 public boolean isStopped() {
107 return false;
108 }
109
110 @Override
111 public Configuration getConfiguration() {
112 return conf;
113 }
114
115 @Override
116 public ZooKeeperWatcher getZooKeeper() {
117 return zkw;
118 }
119
120 @Override
121 public ServerName getServerName() {
122 return null;
123 }
124
125 @Override
126 public CoordinatedStateManager getCoordinatedStateManager() {
127 return cm;
128 }
129
130 @Override
131 public ClusterConnection getConnection() {
132 return null;
133 }
134
135 @Override
136 public MetaTableLocator getMetaTableLocator() {
137 return null;
138 }
139 }
140
141 private void waitForCounter(AtomicLong ctr, long oldval, long newval, long timems)
142 throws Exception {
143 assertTrue("ctr=" + ctr.get() + ", oldval=" + oldval + ", newval=" + newval,
144 waitForCounterBoolean(ctr, oldval, newval, timems));
145 }
146
147 private boolean waitForCounterBoolean(final AtomicLong ctr, final long oldval, long newval,
148 long timems) throws Exception {
149
150 return waitForCounterBoolean(ctr, oldval, newval, timems, true);
151 }
152
153 private boolean waitForCounterBoolean(final AtomicLong ctr, final long oldval, final long newval,
154 long timems, boolean failIfTimeout) throws Exception {
155
156 long timeWaited = TEST_UTIL.waitFor(timems, 10, failIfTimeout,
157 new Waiter.Predicate<Exception>() {
158 @Override
159 public boolean evaluate() throws Exception {
160 return (ctr.get() >= newval);
161 }
162 });
163
164 if( timeWaited > 0) {
165
166 assertEquals(newval, ctr.get());
167 }
168 return true;
169 }
170
171 @Before
172 public void setup() throws Exception {
173 TEST_UTIL.startMiniZKCluster();
174 Configuration conf = TEST_UTIL.getConfiguration();
175 zkw = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
176 "split-log-worker-tests", null);
177 ds = new DummyServer(zkw, conf);
178 ZKUtil.deleteChildrenRecursively(zkw, zkw.baseZNode);
179 ZKUtil.createAndFailSilent(zkw, zkw.baseZNode);
180 assertThat(ZKUtil.checkExists(zkw, zkw.baseZNode), not (is(-1)));
181 LOG.debug(zkw.baseZNode + " created");
182 ZKUtil.createAndFailSilent(zkw, zkw.splitLogZNode);
183 assertThat(ZKUtil.checkExists(zkw, zkw.splitLogZNode), not (is(-1)));
184
185 LOG.debug(zkw.splitLogZNode + " created");
186 ZKUtil.createAndFailSilent(zkw, zkw.rsZNode);
187 assertThat(ZKUtil.checkExists(zkw, zkw.rsZNode), not (is(-1)));
188
189 SplitLogCounters.resetCounters();
190 executorService = new ExecutorService("TestSplitLogWorker");
191 executorService.startExecutorService(ExecutorType.RS_LOG_REPLAY_OPS, 10);
192 this.mode = (conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY, false) ?
193 RecoveryMode.LOG_REPLAY : RecoveryMode.LOG_SPLITTING);
194 }
195
196 @After
197 public void teardown() throws Exception {
198 if (executorService != null) {
199 executorService.shutdown();
200 }
201 TEST_UTIL.shutdownMiniZKCluster();
202 }
203
204 SplitLogWorker.TaskExecutor neverEndingTask =
205 new SplitLogWorker.TaskExecutor() {
206
207 @Override
208 public Status exec(String name, RecoveryMode mode, CancelableProgressable p) {
209 while (true) {
210 try {
211 Thread.sleep(1000);
212 } catch (InterruptedException e) {
213 return Status.PREEMPTED;
214 }
215 if (!p.progress()) {
216 return Status.PREEMPTED;
217 }
218 }
219 }
220
221 };
222
223 @Test(timeout=60000)
224 public void testAcquireTaskAtStartup() throws Exception {
225 LOG.info("testAcquireTaskAtStartup");
226 SplitLogCounters.resetCounters();
227 final String TATAS = "tatas";
228 final ServerName RS = ServerName.valueOf("rs,1,1");
229 RegionServerServices mockedRS = getRegionServer(RS);
230 zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS),
231 new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
232 Ids.OPEN_ACL_UNSAFE,
233 CreateMode.PERSISTENT);
234
235 SplitLogWorker slw =
236 new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS, neverEndingTask);
237 slw.start();
238 try {
239 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
240 byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS));
241 SplitLogTask slt = SplitLogTask.parseFrom(bytes);
242 assertTrue(slt.isOwned(RS));
243 } finally {
244 stopSplitLogWorker(slw);
245 }
246 }
247
248 private void stopSplitLogWorker(final SplitLogWorker slw)
249 throws InterruptedException {
250 if (slw != null) {
251 slw.stop();
252 slw.worker.join(WAIT_TIME);
253 if (slw.worker.isAlive()) {
254 assertTrue(("Could not stop the worker thread slw=" + slw) == null);
255 }
256 }
257 }
258
259 @Test(timeout=60000)
260 public void testRaceForTask() throws Exception {
261 LOG.info("testRaceForTask");
262 SplitLogCounters.resetCounters();
263 final String TRFT = "trft";
264 final ServerName SVR1 = ServerName.valueOf("svr1,1,1");
265 final ServerName SVR2 = ServerName.valueOf("svr2,1,1");
266 zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TRFT),
267 new SplitLogTask.Unassigned(MANAGER, this.mode).toByteArray(),
268 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
269 RegionServerServices mockedRS1 = getRegionServer(SVR1);
270 RegionServerServices mockedRS2 = getRegionServer(SVR2);
271 SplitLogWorker slw1 =
272 new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS1, neverEndingTask);
273 SplitLogWorker slw2 =
274 new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS2, neverEndingTask);
275 slw1.start();
276 slw2.start();
277 try {
278 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
279
280
281 assertTrue(waitForCounterBoolean(SplitLogCounters.tot_wkr_failed_to_grab_task_owned, 0, 1,
282 WAIT_TIME, false) ||
283 SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.get() == 1);
284 byte [] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TRFT));
285 SplitLogTask slt = SplitLogTask.parseFrom(bytes);
286 assertTrue(slt.isOwned(SVR1) || slt.isOwned(SVR2));
287 } finally {
288 stopSplitLogWorker(slw1);
289 stopSplitLogWorker(slw2);
290 }
291 }
292
293 @Test(timeout=60000)
294 public void testPreemptTask() throws Exception {
295 LOG.info("testPreemptTask");
296 SplitLogCounters.resetCounters();
297 final ServerName SRV = ServerName.valueOf("tpt_svr,1,1");
298 final String PATH = ZKSplitLog.getEncodedNodeName(zkw, "tpt_task");
299 RegionServerServices mockedRS = getRegionServer(SRV);
300 SplitLogWorker slw =
301 new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS, neverEndingTask);
302 slw.start();
303 try {
304 Thread.yield();
305 Thread.sleep(1000);
306 waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, WAIT_TIME);
307
308
309 zkw.getRecoverableZooKeeper().create(PATH,
310 new SplitLogTask.Unassigned(MANAGER, this.mode).toByteArray(),
311 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
312
313 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
314 assertEquals(1, slw.getTaskReadySeq());
315 byte [] bytes = ZKUtil.getData(zkw, PATH);
316 SplitLogTask slt = SplitLogTask.parseFrom(bytes);
317 assertTrue(slt.isOwned(SRV));
318 slt = new SplitLogTask.Owned(MANAGER, this.mode);
319 ZKUtil.setData(zkw, PATH, slt.toByteArray());
320 waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME);
321 } finally {
322 stopSplitLogWorker(slw);
323 }
324 }
325
326 @Test(timeout=60000)
327 public void testMultipleTasks() throws Exception {
328 LOG.info("testMultipleTasks");
329 SplitLogCounters.resetCounters();
330 final ServerName SRV = ServerName.valueOf("tmt_svr,1,1");
331 final String PATH1 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task");
332 RegionServerServices mockedRS = getRegionServer(SRV);
333 SplitLogWorker slw =
334 new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS, neverEndingTask);
335 slw.start();
336 try {
337 Thread.yield();
338 Thread.sleep(100);
339 waitForCounter(SplitLogCounters.tot_wkr_task_grabing, 0, 1, WAIT_TIME);
340
341 SplitLogTask unassignedManager =
342 new SplitLogTask.Unassigned(MANAGER, this.mode);
343 zkw.getRecoverableZooKeeper().create(PATH1, unassignedManager.toByteArray(),
344 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
345
346 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
347
348
349
350 final String PATH2 = ZKSplitLog.getEncodedNodeName(zkw, "tmt_task_2");
351 zkw.getRecoverableZooKeeper().create(PATH2, unassignedManager.toByteArray(),
352 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
353
354
355 final ServerName anotherWorker = ServerName.valueOf("another-worker,1,1");
356 SplitLogTask slt = new SplitLogTask.Owned(anotherWorker, this.mode);
357 ZKUtil.setData(zkw, PATH1, slt.toByteArray());
358 waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME);
359
360 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, WAIT_TIME);
361 assertEquals(2, slw.getTaskReadySeq());
362 byte [] bytes = ZKUtil.getData(zkw, PATH2);
363 slt = SplitLogTask.parseFrom(bytes);
364 assertTrue(slt.isOwned(SRV));
365 } finally {
366 stopSplitLogWorker(slw);
367 }
368 }
369
370 @Test(timeout=60000)
371 public void testRescan() throws Exception {
372 LOG.info("testRescan");
373 SplitLogCounters.resetCounters();
374 final ServerName SRV = ServerName.valueOf("svr,1,1");
375 RegionServerServices mockedRS = getRegionServer(SRV);
376 slw = new SplitLogWorker(ds, TEST_UTIL.getConfiguration(), mockedRS, neverEndingTask);
377 slw.start();
378 Thread.yield();
379 Thread.sleep(100);
380
381 String task = ZKSplitLog.getEncodedNodeName(zkw, "task");
382 SplitLogTask slt = new SplitLogTask.Unassigned(MANAGER, this.mode);
383 zkw.getRecoverableZooKeeper().create(task,slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
384 CreateMode.PERSISTENT);
385
386 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 1, WAIT_TIME);
387
388
389
390 ZKUtil.setData(zkw, task, slt.toByteArray());
391 waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 0, 1, WAIT_TIME);
392
393
394 String rescan = ZKSplitLog.getEncodedNodeName(zkw, "RESCAN");
395 rescan = zkw.getRecoverableZooKeeper().create(rescan, slt.toByteArray(), Ids.OPEN_ACL_UNSAFE,
396 CreateMode.PERSISTENT_SEQUENTIAL);
397
398 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 1, 2, WAIT_TIME);
399
400
401
402 ZKUtil.setData(zkw, task, slt.toByteArray());
403 waitForCounter(SplitLogCounters.tot_wkr_preempt_task, 1, 2, WAIT_TIME);
404 waitForCounter(SplitLogCounters.tot_wkr_task_acquired_rescan, 0, 1, WAIT_TIME);
405
406 List<String> nodes = ZKUtil.listChildrenNoWatch(zkw, zkw.splitLogZNode);
407 LOG.debug(nodes);
408 int num = 0;
409 for (String node : nodes) {
410 num++;
411 if (node.startsWith("RESCAN")) {
412 String name = ZKSplitLog.getEncodedNodeName(zkw, node);
413 String fn = ZKSplitLog.getFileName(name);
414 byte [] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(zkw.splitLogZNode, fn));
415 slt = SplitLogTask.parseFrom(data);
416 assertTrue(slt.toString(), slt.isDone(SRV));
417 }
418 }
419 assertEquals(2, num);
420 }
421
422 @Test(timeout=60000)
423 public void testAcquireMultiTasks() throws Exception {
424 LOG.info("testAcquireMultiTasks");
425 SplitLogCounters.resetCounters();
426 final String TATAS = "tatas";
427 final ServerName RS = ServerName.valueOf("rs,1,1");
428 final int maxTasks = 3;
429 Configuration testConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
430 testConf.setInt("hbase.regionserver.wal.max.splitters", maxTasks);
431 RegionServerServices mockedRS = getRegionServer(RS);
432 for (int i = 0; i < maxTasks; i++) {
433 zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS + i),
434 new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
435 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
436 }
437
438 SplitLogWorker slw = new SplitLogWorker(ds, testConf, mockedRS, neverEndingTask);
439 slw.start();
440 try {
441 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, maxTasks, WAIT_TIME);
442 for (int i = 0; i < maxTasks; i++) {
443 byte[] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i));
444 SplitLogTask slt = SplitLogTask.parseFrom(bytes);
445 assertTrue(slt.isOwned(RS));
446 }
447 } finally {
448 stopSplitLogWorker(slw);
449 }
450 }
451
452
453
454
455
456
457 @Test(timeout=60000)
458 public void testAcquireMultiTasksByAvgTasksPerRS() throws Exception {
459 LOG.info("testAcquireMultiTasks");
460 SplitLogCounters.resetCounters();
461 final String TATAS = "tatas";
462 final ServerName RS = ServerName.valueOf("rs,1,1");
463 final ServerName RS2 = ServerName.valueOf("rs,1,2");
464 final int maxTasks = 3;
465 Configuration testConf = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
466 testConf.setInt("hbase.regionserver.wal.max.splitters", maxTasks);
467 RegionServerServices mockedRS = getRegionServer(RS);
468
469
470 String rsPath = ZKUtil.joinZNode(zkw.rsZNode, RS.getServerName());
471 zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
472 rsPath = ZKUtil.joinZNode(zkw.rsZNode, RS2.getServerName());
473 zkw.getRecoverableZooKeeper().create(rsPath, null, Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
474
475 for (int i = 0; i < maxTasks; i++) {
476 zkw.getRecoverableZooKeeper().create(ZKSplitLog.getEncodedNodeName(zkw, TATAS + i),
477 new SplitLogTask.Unassigned(ServerName.valueOf("mgr,1,1"), this.mode).toByteArray(),
478 Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
479 }
480
481 SplitLogWorker slw = new SplitLogWorker(ds, testConf, mockedRS, neverEndingTask);
482 slw.start();
483 try {
484 int acquiredTasks = 0;
485 waitForCounter(SplitLogCounters.tot_wkr_task_acquired, 0, 2, WAIT_TIME);
486 for (int i = 0; i < maxTasks; i++) {
487 byte[] bytes = ZKUtil.getData(zkw, ZKSplitLog.getEncodedNodeName(zkw, TATAS + i));
488 SplitLogTask slt = SplitLogTask.parseFrom(bytes);
489 if (slt.isOwned(RS)) {
490 acquiredTasks++;
491 }
492 }
493 assertEquals(2, acquiredTasks);
494 } finally {
495 stopSplitLogWorker(slw);
496 }
497 }
498
499
500
501
502
503
504 private RegionServerServices getRegionServer(ServerName name) {
505
506 RegionServerServices mockedServer = mock(RegionServerServices.class);
507 when(mockedServer.getConfiguration()).thenReturn(TEST_UTIL.getConfiguration());
508 when(mockedServer.getServerName()).thenReturn(name);
509 when(mockedServer.getZooKeeper()).thenReturn(zkw);
510 when(mockedServer.isStopped()).thenReturn(false);
511 when(mockedServer.getExecutorService()).thenReturn(executorService);
512
513 return mockedServer;
514 }
515
516 }