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