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