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.coordination;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collections;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.concurrent.atomic.AtomicInteger;
28 import java.util.concurrent.atomic.AtomicLong;
29
30 import org.apache.commons.lang.math.RandomUtils;
31 import org.apache.commons.lang.mutable.MutableInt;
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.HConstants;
38 import org.apache.hadoop.hbase.ServerName;
39 import org.apache.hadoop.hbase.SplitLogCounters;
40 import org.apache.hadoop.hbase.SplitLogTask;
41 import org.apache.hadoop.hbase.classification.InterfaceAudience;
42 import org.apache.hadoop.hbase.exceptions.DeserializationException;
43 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
44 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
45 import org.apache.hadoop.hbase.regionserver.Region;
46 import org.apache.hadoop.hbase.regionserver.RegionServerServices;
47 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
48 import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
49 import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
50 import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
51 import org.apache.hadoop.hbase.util.CancelableProgressable;
52 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
53 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
54 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
55 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
56 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
57 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
58 import org.apache.hadoop.util.StringUtils;
59 import org.apache.zookeeper.AsyncCallback;
60 import org.apache.zookeeper.KeeperException;
61 import org.apache.zookeeper.data.Stat;
62
63
64
65
66
67
68 @InterfaceAudience.Private
69 public class ZkSplitLogWorkerCoordination extends ZooKeeperListener implements
70 SplitLogWorkerCoordination {
71
72 private static final Log LOG = LogFactory.getLog(ZkSplitLogWorkerCoordination.class);
73
74 private static final int checkInterval = 5000;
75 private static final int FAILED_TO_OWN_TASK = -1;
76
77 private SplitLogWorker worker;
78
79 private TaskExecutor splitTaskExecutor;
80
81 private final Object taskReadyLock = new Object();
82 private AtomicInteger taskReadySeq = new AtomicInteger(0);
83 private volatile String currentTask = null;
84 private int currentVersion;
85 private volatile boolean shouldStop = false;
86 private final Object grabTaskLock = new Object();
87 private boolean workerInGrabTask = false;
88 private int reportPeriod;
89 private RegionServerServices server = null;
90 protected final AtomicInteger tasksInProgress = new AtomicInteger(0);
91 private int maxConcurrentTasks = 0;
92
93 private final ZkCoordinatedStateManager manager;
94
95 public ZkSplitLogWorkerCoordination(ZkCoordinatedStateManager zkCoordinatedStateManager,
96 ZooKeeperWatcher watcher) {
97 super(watcher);
98 manager = zkCoordinatedStateManager;
99
100 }
101
102
103
104
105 @Override
106 public void nodeChildrenChanged(String path) {
107 if (path.equals(watcher.splitLogZNode)) {
108 if (LOG.isTraceEnabled()) LOG.trace("tasks arrived or departed on " + path);
109 synchronized (taskReadyLock) {
110 this.taskReadySeq.incrementAndGet();
111 taskReadyLock.notify();
112 }
113 }
114 }
115
116
117
118
119 @Override
120 public void nodeDataChanged(String path) {
121
122
123 synchronized (grabTaskLock) {
124 if (workerInGrabTask) {
125
126 String taskpath = currentTask;
127 if (taskpath != null && taskpath.equals(path)) {
128 getDataSetWatchAsync();
129 }
130 }
131 }
132 }
133
134
135
136
137 @Override
138 public void init(RegionServerServices server, Configuration conf,
139 TaskExecutor splitExecutor, SplitLogWorker worker) {
140 this.server = server;
141 this.worker = worker;
142 this.splitTaskExecutor = splitExecutor;
143 maxConcurrentTasks = conf.getInt("hbase.regionserver.wal.max.splitters", DEFAULT_MAX_SPLITTERS);
144 reportPeriod =
145 conf.getInt("hbase.splitlog.report.period",
146 conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
147 ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT) / 3);
148 }
149
150
151
152 void getDataSetWatchFailure(String path) {
153 synchronized (grabTaskLock) {
154 if (workerInGrabTask) {
155
156 String taskpath = currentTask;
157 if (taskpath != null && taskpath.equals(path)) {
158 LOG.info("retrying data watch on " + path);
159 SplitLogCounters.tot_wkr_get_data_retry.incrementAndGet();
160 getDataSetWatchAsync();
161 } else {
162
163
164 }
165 }
166 }
167 }
168
169 public void getDataSetWatchAsync() {
170 watcher.getRecoverableZooKeeper().getZooKeeper()
171 .getData(currentTask, watcher, new GetDataAsyncCallback(), null);
172 SplitLogCounters.tot_wkr_get_data_queued.incrementAndGet();
173 }
174
175 void getDataSetWatchSuccess(String path, byte[] data) {
176 SplitLogTask slt;
177 try {
178 slt = SplitLogTask.parseFrom(data);
179 } catch (DeserializationException e) {
180 LOG.warn("Failed parse", e);
181 return;
182 }
183 synchronized (grabTaskLock) {
184 if (workerInGrabTask) {
185
186 String taskpath = currentTask;
187 if (taskpath != null && taskpath.equals(path)) {
188 ServerName serverName = manager.getServer().getServerName();
189
190
191
192
193
194
195 if (!slt.isOwned(serverName) && !slt.isDone(serverName) && !slt.isErr(serverName)
196 && !slt.isResigned(serverName)) {
197 LOG.info("task " + taskpath + " preempted from " + serverName
198 + ", current task state and owner=" + slt.toString());
199 worker.stopTask();
200 }
201 }
202 }
203 }
204 }
205
206
207
208
209
210
211
212 private boolean grabTask(String path) {
213 Stat stat = new Stat();
214 byte[] data;
215 synchronized (grabTaskLock) {
216 currentTask = path;
217 workerInGrabTask = true;
218 if (Thread.interrupted()) {
219 return false;
220 }
221 }
222 try {
223 try {
224 if ((data = ZKUtil.getDataNoWatch(watcher, path, stat)) == null) {
225 SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
226 return false;
227 }
228 } catch (KeeperException e) {
229 LOG.warn("Failed to get data for znode " + path, e);
230 SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
231 return false;
232 }
233 SplitLogTask slt;
234 try {
235 slt = SplitLogTask.parseFrom(data);
236 } catch (DeserializationException e) {
237 LOG.warn("Failed parse data for znode " + path, e);
238 SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
239 return false;
240 }
241 if (!slt.isUnassigned()) {
242 SplitLogCounters.tot_wkr_failed_to_grab_task_owned.incrementAndGet();
243 return false;
244 }
245
246 currentVersion =
247 attemptToOwnTask(true, watcher, server.getServerName(), path,
248 slt.getMode(), stat.getVersion());
249 if (currentVersion < 0) {
250 SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
251 return false;
252 }
253
254 if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
255 ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
256 new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
257 splitTaskDetails.setTaskNode(currentTask);
258 splitTaskDetails.setCurTaskZKVersion(new MutableInt(currentVersion));
259
260 endTask(new SplitLogTask.Done(server.getServerName(), slt.getMode()),
261 SplitLogCounters.tot_wkr_task_acquired_rescan, splitTaskDetails);
262 return false;
263 }
264
265 LOG.info("worker " + server.getServerName() + " acquired task " + path);
266 SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
267 getDataSetWatchAsync();
268
269 submitTask(path, slt.getMode(), currentVersion, reportPeriod);
270
271
272 try {
273 int sleepTime = RandomUtils.nextInt(500) + 500;
274 Thread.sleep(sleepTime);
275 } catch (InterruptedException e) {
276 LOG.warn("Interrupted while yielding for other region servers", e);
277 Thread.currentThread().interrupt();
278 }
279 return true;
280 } finally {
281 synchronized (grabTaskLock) {
282 workerInGrabTask = false;
283
284
285 Thread.interrupted();
286 }
287 }
288 }
289
290
291
292
293
294
295 void submitTask(final String curTask, final RecoveryMode mode, final int curTaskZKVersion,
296 final int reportPeriod) {
297 final MutableInt zkVersion = new MutableInt(curTaskZKVersion);
298
299 CancelableProgressable reporter = new CancelableProgressable() {
300 private long last_report_at = 0;
301
302 @Override
303 public boolean progress() {
304 long t = EnvironmentEdgeManager.currentTime();
305 if ((t - last_report_at) > reportPeriod) {
306 last_report_at = t;
307 int latestZKVersion =
308 attemptToOwnTask(false, watcher, server.getServerName(), curTask,
309 mode, zkVersion.intValue());
310 if (latestZKVersion < 0) {
311 LOG.warn("Failed to heartbeat the task" + curTask);
312 return false;
313 }
314 zkVersion.setValue(latestZKVersion);
315 }
316 return true;
317 }
318 };
319 ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
320 new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
321 splitTaskDetails.setTaskNode(curTask);
322 splitTaskDetails.setCurTaskZKVersion(zkVersion);
323
324 WALSplitterHandler hsh =
325 new WALSplitterHandler(server, this, splitTaskDetails, reporter,
326 this.tasksInProgress, splitTaskExecutor, mode);
327 server.getExecutorService().submit(hsh);
328 }
329
330
331
332
333 private boolean areSplittersAvailable() {
334 return maxConcurrentTasks - tasksInProgress.get() > 0;
335 }
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350 protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw,
351 ServerName server, String task, RecoveryMode mode, int taskZKVersion) {
352 int latestZKVersion = FAILED_TO_OWN_TASK;
353 try {
354 SplitLogTask slt = new SplitLogTask.Owned(server, mode);
355 Stat stat = zkw.getRecoverableZooKeeper().setData(task, slt.toByteArray(), taskZKVersion);
356 if (stat == null) {
357 LOG.warn("zk.setData() returned null for path " + task);
358 SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
359 return FAILED_TO_OWN_TASK;
360 }
361 latestZKVersion = stat.getVersion();
362 SplitLogCounters.tot_wkr_task_heartbeat.incrementAndGet();
363 return latestZKVersion;
364 } catch (KeeperException e) {
365 if (!isFirstTime) {
366 if (e.code().equals(KeeperException.Code.NONODE)) {
367 LOG.warn("NONODE failed to assert ownership for " + task, e);
368 } else if (e.code().equals(KeeperException.Code.BADVERSION)) {
369 LOG.warn("BADVERSION failed to assert ownership for " + task, e);
370 } else {
371 LOG.warn("failed to assert ownership for " + task, e);
372 }
373 }
374 } catch (InterruptedException e1) {
375 LOG.warn("Interrupted while trying to assert ownership of " + task + " "
376 + StringUtils.stringifyException(e1));
377 Thread.currentThread().interrupt();
378 }
379 SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
380 return FAILED_TO_OWN_TASK;
381 }
382
383
384
385
386
387
388
389
390
391
392 @Override
393 public void taskLoop() throws InterruptedException {
394 while (!shouldStop) {
395 int seq_start = taskReadySeq.get();
396 List<String> paths = null;
397 paths = getTaskList();
398 if (paths == null) {
399 LOG.warn("Could not get tasks, did someone remove " + watcher.splitLogZNode
400 + " ... worker thread exiting.");
401 return;
402 }
403
404
405 Collections.shuffle(paths);
406
407 int offset = 0;
408 for (int i = 0; i < paths.size(); i++) {
409 if (DefaultWALProvider.isMetaFile(paths.get(i))) {
410 offset = i;
411 break;
412 }
413 }
414 int numTasks = paths.size();
415 boolean taskGrabbed = false;
416 for (int i = 0; i < numTasks; i++) {
417 while (!shouldStop) {
418 if (this.areSplittersAvailable()) {
419 if (LOG.isTraceEnabled()) {
420 LOG.trace("Current region server " + server.getServerName()
421 + " is ready to take more tasks, will get task list and try grab tasks again.");
422 }
423 int idx = (i + offset) % paths.size();
424
425
426 taskGrabbed |= grabTask(ZKUtil.joinZNode(watcher.splitLogZNode,
427 paths.get(idx)));
428 break;
429 } else {
430 if (LOG.isTraceEnabled()) {
431 LOG.trace("Current region server " + server.getServerName()
432 + " has " + this.tasksInProgress.get()
433 + " tasks in progress and can't take more.");
434 }
435 Thread.sleep(100);
436 }
437 }
438 if (shouldStop) {
439 return;
440 }
441 }
442 if (!taskGrabbed && !shouldStop) {
443
444 Thread.sleep(1000);
445 }
446 SplitLogCounters.tot_wkr_task_grabing.incrementAndGet();
447 synchronized (taskReadyLock) {
448 while (seq_start == taskReadySeq.get()) {
449 taskReadyLock.wait(checkInterval);
450 if (server != null) {
451
452 Map<String, Region> recoveringRegions = server.getRecoveringRegions();
453 if (!recoveringRegions.isEmpty()) {
454
455
456 List<String> tmpCopy = new ArrayList<String>(recoveringRegions.keySet());
457 int listSize = tmpCopy.size();
458 for (int i = 0; i < listSize; i++) {
459 String region = tmpCopy.get(i);
460 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
461 try {
462 if (ZKUtil.checkExists(watcher, nodePath) == -1) {
463 server.getExecutorService().submit(
464 new FinishRegionRecoveringHandler(server, region, nodePath));
465 } else {
466
467
468
469
470
471 break;
472 }
473 } catch (KeeperException e) {
474
475 LOG.debug("Got a zookeeper when trying to open a recovering region", e);
476 break;
477 }
478 }
479 }
480 }
481 }
482 }
483 }
484 }
485
486 private List<String> getTaskList() throws InterruptedException {
487 List<String> childrenPaths = null;
488 long sleepTime = 1000;
489
490
491 while (!shouldStop) {
492 try {
493 childrenPaths =
494 ZKUtil.listChildrenAndWatchForNewChildren(watcher,
495 watcher.splitLogZNode);
496 if (childrenPaths != null) {
497 return childrenPaths;
498 }
499 } catch (KeeperException e) {
500 LOG.warn("Could not get children of znode " + watcher.splitLogZNode, e);
501 }
502 LOG.debug("Retry listChildren of znode " + watcher.splitLogZNode
503 + " after sleep for " + sleepTime + "ms!");
504 Thread.sleep(sleepTime);
505 }
506 return childrenPaths;
507 }
508
509 @Override
510 public void markCorrupted(Path rootDir, String name, FileSystem fs) {
511 ZKSplitLog.markCorrupted(rootDir, name, fs);
512 }
513
514 @Override
515 public boolean isReady() throws InterruptedException {
516 int result = -1;
517 try {
518 result = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
519 } catch (KeeperException e) {
520
521 LOG.warn("Exception when checking for " + watcher.splitLogZNode
522 + " ... retrying", e);
523 }
524 if (result == -1) {
525 LOG.info(watcher.splitLogZNode
526 + " znode does not exist, waiting for master to create");
527 Thread.sleep(1000);
528 }
529 return (result != -1);
530 }
531
532 @Override
533 public int getTaskReadySeq() {
534 return taskReadySeq.get();
535 }
536
537 @Override
538 public void registerListener() {
539 watcher.registerListener(this);
540 }
541
542 @Override
543 public void removeListener() {
544 watcher.unregisterListener(this);
545 }
546
547
548 @Override
549 public void stopProcessingTasks() {
550 this.shouldStop = true;
551
552 }
553
554 @Override
555 public boolean isStop() {
556 return shouldStop;
557 }
558
559 @Override
560 public RegionStoreSequenceIds getRegionFlushedSequenceId(String failedServerName, String key)
561 throws IOException {
562 return ZKSplitLog.getRegionFlushedSequenceId(watcher, failedServerName, key);
563 }
564
565
566
567
568 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
569 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
570
571 @Override
572 public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
573 SplitLogCounters.tot_wkr_get_data_result.incrementAndGet();
574 if (rc != 0) {
575 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
576 getDataSetWatchFailure(path);
577 return;
578 }
579 data = watcher.getRecoverableZooKeeper().removeMetaData(data);
580 getDataSetWatchSuccess(path, data);
581 }
582 }
583
584
585
586
587
588
589
590
591
592
593 @Override
594 public void endTask(SplitLogTask slt, AtomicLong ctr, SplitTaskDetails details) {
595 ZkSplitTaskDetails zkDetails = (ZkSplitTaskDetails) details;
596 String task = zkDetails.getTaskNode();
597 int taskZKVersion = zkDetails.getCurTaskZKVersion().intValue();
598 try {
599 if (ZKUtil.setData(watcher, task, slt.toByteArray(), taskZKVersion)) {
600 LOG.info("successfully transitioned task " + task + " to final state " + slt);
601 ctr.incrementAndGet();
602 return;
603 }
604 LOG.warn("failed to transistion task " + task + " to end state " + slt
605 + " because of version mismatch ");
606 } catch (KeeperException.BadVersionException bve) {
607 LOG.warn("transisition task " + task + " to " + slt + " failed because of version mismatch",
608 bve);
609 } catch (KeeperException.NoNodeException e) {
610 LOG.fatal(
611 "logic error - end task " + task + " " + slt + " failed because task doesn't exist", e);
612 } catch (KeeperException e) {
613 LOG.warn("failed to end task, " + task + " " + slt, e);
614 }
615 SplitLogCounters.tot_wkr_final_transition_failed.incrementAndGet();
616 }
617
618
619
620
621
622 public static class ZkSplitTaskDetails implements SplitTaskDetails {
623 private String taskNode;
624 private MutableInt curTaskZKVersion;
625
626 public ZkSplitTaskDetails() {
627 }
628
629 public ZkSplitTaskDetails(String taskNode, MutableInt curTaskZKVersion) {
630 this.taskNode = taskNode;
631 this.curTaskZKVersion = curTaskZKVersion;
632 }
633
634 public String getTaskNode() {
635 return taskNode;
636 }
637
638 public void setTaskNode(String taskNode) {
639 this.taskNode = taskNode;
640 }
641
642 public MutableInt getCurTaskZKVersion() {
643 return curTaskZKVersion;
644 }
645
646 public void setCurTaskZKVersion(MutableInt curTaskZKVersion) {
647 this.curTaskZKVersion = curTaskZKVersion;
648 }
649
650 @Override
651 public String getWALFile() {
652 return ZKSplitLog.getFileName(taskNode);
653 }
654 }
655
656 }