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