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 = this.conf.getBoolean(
178 HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
179 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
180 if (distributedLogReplay) {
181
182 HConnectionManager.getConnection(conf);
183 }
184
185
186 int res = -1;
187 while (res == -1 && !exitWorker) {
188 try {
189 res = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
190 } catch (KeeperException e) {
191
192 LOG.warn("Exception when checking for " + watcher.splitLogZNode + " ... retrying", e);
193 }
194 if (res == -1) {
195 try {
196 LOG.info(watcher.splitLogZNode + " znode does not exist, waiting for master to create");
197 Thread.sleep(1000);
198 } catch (InterruptedException e) {
199 LOG.debug("Interrupted while waiting for " + watcher.splitLogZNode
200 + (exitWorker ? "" : " (ERROR: exitWorker is not set, " +
201 "exiting anyway)"));
202 exitWorker = true;
203 break;
204 }
205 }
206 }
207
208 if (!exitWorker) {
209 taskLoop();
210 }
211 } catch (Throwable t) {
212
213
214 LOG.error("unexpected error ", t);
215 } finally {
216 LOG.info("SplitLogWorker " + this.serverName + " exiting");
217 }
218 }
219
220
221
222
223
224
225
226
227
228 private void taskLoop() {
229 while (!exitWorker) {
230 int seq_start = taskReadySeq;
231 List<String> paths = getTaskList();
232 if (paths == null) {
233 LOG.warn("Could not get tasks, did someone remove " +
234 this.watcher.splitLogZNode + " ... worker thread exiting.");
235 return;
236 }
237
238 int offset = (int) (Math.random() * paths.size());
239 for(int i = 0; i < paths.size(); i ++){
240 if(HLogUtil.isMetaFile(paths.get(i))) {
241 offset = i;
242 break;
243 }
244 }
245 int numTasks = paths.size();
246 for (int i = 0; i < numTasks; i++) {
247 int idx = (i + offset) % paths.size();
248
249
250 if (this.calculateAvailableSplitters(numTasks) > 0) {
251 grabTask(ZKUtil.joinZNode(watcher.splitLogZNode, paths.get(idx)));
252 } else {
253 LOG.debug("Current region server " + this.serverName + " has "
254 + this.tasksInProgress.get() + " tasks in progress and can't take more.");
255 break;
256 }
257 if (exitWorker) {
258 return;
259 }
260 }
261 SplitLogCounters.tot_wkr_task_grabing.incrementAndGet();
262 synchronized (taskReadyLock) {
263 while (seq_start == taskReadySeq) {
264 try {
265 taskReadyLock.wait(checkInterval);
266 if (this.server != null) {
267
268 Map<String, HRegion> recoveringRegions = this.server.getRecoveringRegions();
269 if (!recoveringRegions.isEmpty()) {
270
271
272 List<String> tmpCopy = new ArrayList<String>(recoveringRegions.keySet());
273 for (String region : tmpCopy) {
274 String nodePath = ZKUtil.joinZNode(this.watcher.recoveringRegionsZNode, region);
275 try {
276 if (ZKUtil.checkExists(this.watcher, nodePath) == -1) {
277 HRegion r = recoveringRegions.remove(region);
278 if (r != null) {
279 r.setRecovering(false);
280 }
281 LOG.debug("Mark recovering region:" + region + " up.");
282 } else {
283
284
285
286
287
288 break;
289 }
290 } catch (KeeperException e) {
291
292 LOG.debug("Got a zookeeper when trying to open a recovering region", e);
293 break;
294 }
295 }
296 }
297 }
298 } catch (InterruptedException e) {
299 LOG.info("SplitLogWorker interrupted while waiting for task," +
300 " exiting: " + e.toString() + (exitWorker ? "" :
301 " (ERROR: exitWorker is not set, exiting anyway)"));
302 exitWorker = true;
303 return;
304 }
305 }
306 }
307
308 }
309 }
310
311
312
313
314
315
316 private void grabTask(String path) {
317 Stat stat = new Stat();
318 long t = -1;
319 byte[] data;
320 synchronized (grabTaskLock) {
321 currentTask = path;
322 workerInGrabTask = true;
323 if (Thread.interrupted()) {
324 return;
325 }
326 }
327 try {
328 try {
329 if ((data = ZKUtil.getDataNoWatch(this.watcher, path, stat)) == null) {
330 SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
331 return;
332 }
333 } catch (KeeperException e) {
334 LOG.warn("Failed to get data for znode " + path, e);
335 SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
336 return;
337 }
338 SplitLogTask slt;
339 try {
340 slt = SplitLogTask.parseFrom(data);
341 } catch (DeserializationException e) {
342 LOG.warn("Failed parse data for znode " + path, e);
343 SplitLogCounters.tot_wkr_failed_to_grab_task_exception.incrementAndGet();
344 return;
345 }
346 if (!slt.isUnassigned()) {
347 SplitLogCounters.tot_wkr_failed_to_grab_task_owned.incrementAndGet();
348 return;
349 }
350
351 currentVersion = attemptToOwnTask(true, watcher, serverName, path, stat.getVersion());
352 if (currentVersion < 0) {
353 SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
354 return;
355 }
356
357 if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
358 HLogSplitterHandler.endTask(watcher, new SplitLogTask.Done(this.serverName),
359 SplitLogCounters.tot_wkr_task_acquired_rescan, currentTask, currentVersion);
360 return;
361 }
362
363 LOG.info("worker " + serverName + " acquired task " + path);
364 SplitLogCounters.tot_wkr_task_acquired.incrementAndGet();
365 getDataSetWatchAsync();
366
367 submitTask(path, currentVersion, this.report_period);
368
369
370 try {
371 int sleepTime = RandomUtils.nextInt(500) + 500;
372 Thread.sleep(sleepTime);
373 } catch (InterruptedException e) {
374 LOG.warn("Interrupted while yielding for other region servers", e);
375 Thread.currentThread().interrupt();
376 }
377 } finally {
378 synchronized (grabTaskLock) {
379 workerInGrabTask = false;
380
381
382 Thread.interrupted();
383 }
384 }
385 }
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401 protected static int attemptToOwnTask(boolean isFirstTime, ZooKeeperWatcher zkw,
402 ServerName server, String task, int taskZKVersion) {
403 int latestZKVersion = FAILED_TO_OWN_TASK;
404 try {
405 SplitLogTask slt = new SplitLogTask.Owned(server);
406 Stat stat = zkw.getRecoverableZooKeeper().setData(task, slt.toByteArray(), taskZKVersion);
407 if (stat == null) {
408 LOG.warn("zk.setData() returned null for path " + task);
409 SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
410 return FAILED_TO_OWN_TASK;
411 }
412 latestZKVersion = stat.getVersion();
413 SplitLogCounters.tot_wkr_task_heartbeat.incrementAndGet();
414 return latestZKVersion;
415 } catch (KeeperException e) {
416 if (!isFirstTime) {
417 if (e.code().equals(KeeperException.Code.NONODE)) {
418 LOG.warn("NONODE failed to assert ownership for " + task, e);
419 } else if (e.code().equals(KeeperException.Code.BADVERSION)) {
420 LOG.warn("BADVERSION failed to assert ownership for " + task, e);
421 } else {
422 LOG.warn("failed to assert ownership for " + task, e);
423 }
424 }
425 } catch (InterruptedException e1) {
426 LOG.warn("Interrupted while trying to assert ownership of " +
427 task + " " + StringUtils.stringifyException(e1));
428 Thread.currentThread().interrupt();
429 }
430 SplitLogCounters.tot_wkr_task_heartbeat_failed.incrementAndGet();
431 return FAILED_TO_OWN_TASK;
432 }
433
434
435
436
437
438
439
440
441 private int calculateAvailableSplitters(int numTasks) {
442
443 int availableRSs = 1;
444 try {
445 List<String> regionServers = ZKUtil.listChildrenNoWatch(watcher, watcher.rsZNode);
446 availableRSs = Math.max(availableRSs, (regionServers == null) ? 0 : regionServers.size());
447 } catch (KeeperException e) {
448
449 LOG.debug("getAvailableRegionServers got ZooKeeper exception", e);
450 }
451
452 int expectedTasksPerRS = (numTasks / availableRSs) + ((numTasks % availableRSs == 0) ? 0 : 1);
453 expectedTasksPerRS = Math.max(1, expectedTasksPerRS);
454
455 return Math.min(expectedTasksPerRS, this.maxConcurrentTasks) - this.tasksInProgress.get();
456 }
457
458
459
460
461
462
463 void submitTask(final String curTask, final int curTaskZKVersion, final int reportPeriod) {
464 final MutableInt zkVersion = new MutableInt(curTaskZKVersion);
465
466 CancelableProgressable reporter = new CancelableProgressable() {
467 private long last_report_at = 0;
468
469 @Override
470 public boolean progress() {
471 long t = EnvironmentEdgeManager.currentTimeMillis();
472 if ((t - last_report_at) > reportPeriod) {
473 last_report_at = t;
474 int latestZKVersion =
475 attemptToOwnTask(false, watcher, serverName, curTask, zkVersion.intValue());
476 if (latestZKVersion < 0) {
477 LOG.warn("Failed to heartbeat the task" + curTask);
478 return false;
479 }
480 zkVersion.setValue(latestZKVersion);
481 }
482 return true;
483 }
484 };
485
486 HLogSplitterHandler hsh =
487 new HLogSplitterHandler(this.server, curTask, zkVersion, reporter, this.tasksInProgress,
488 this.splitTaskExecutor);
489 this.executorService.submit(hsh);
490 }
491
492 void getDataSetWatchAsync() {
493 this.watcher.getRecoverableZooKeeper().getZooKeeper().
494 getData(currentTask, this.watcher,
495 new GetDataAsyncCallback(), null);
496 SplitLogCounters.tot_wkr_get_data_queued.incrementAndGet();
497 }
498
499 void getDataSetWatchSuccess(String path, byte[] data) {
500 SplitLogTask slt;
501 try {
502 slt = SplitLogTask.parseFrom(data);
503 } catch (DeserializationException e) {
504 LOG.warn("Failed parse", e);
505 return;
506 }
507 synchronized (grabTaskLock) {
508 if (workerInGrabTask) {
509
510 String taskpath = currentTask;
511 if (taskpath != null && taskpath.equals(path)) {
512
513
514
515
516
517
518 if (! slt.isOwned(this.serverName) &&
519 ! slt.isDone(this.serverName) &&
520 ! slt.isErr(this.serverName) &&
521 ! slt.isResigned(this.serverName)) {
522 LOG.info("task " + taskpath + " preempted from " +
523 serverName + ", current task state and owner=" + slt.toString());
524 stopTask();
525 }
526 }
527 }
528 }
529 }
530
531 void getDataSetWatchFailure(String path) {
532 synchronized (grabTaskLock) {
533 if (workerInGrabTask) {
534
535 String taskpath = currentTask;
536 if (taskpath != null && taskpath.equals(path)) {
537 LOG.info("retrying data watch on " + path);
538 SplitLogCounters.tot_wkr_get_data_retry.incrementAndGet();
539 getDataSetWatchAsync();
540 } else {
541
542
543 }
544 }
545 }
546 }
547
548 @Override
549 public void nodeDataChanged(String path) {
550
551
552 synchronized (grabTaskLock) {
553 if (workerInGrabTask) {
554
555 String taskpath = currentTask;
556 if (taskpath!= null && taskpath.equals(path)) {
557 getDataSetWatchAsync();
558 }
559 }
560 }
561 }
562
563
564 private List<String> getTaskList() {
565 List<String> childrenPaths = null;
566 long sleepTime = 1000;
567
568
569 while (!exitWorker) {
570 try {
571 childrenPaths = ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
572 this.watcher.splitLogZNode);
573 if (childrenPaths != null) {
574 return childrenPaths;
575 }
576 } catch (KeeperException e) {
577 LOG.warn("Could not get children of znode "
578 + this.watcher.splitLogZNode, e);
579 }
580 try {
581 LOG.debug("Retry listChildren of znode " + this.watcher.splitLogZNode
582 + " after sleep for " + sleepTime + "ms!");
583 Thread.sleep(sleepTime);
584 } catch (InterruptedException e1) {
585 LOG.warn("Interrupted while trying to get task list ...", e1);
586 Thread.currentThread().interrupt();
587 }
588 }
589 return childrenPaths;
590 }
591
592 @Override
593 public void nodeChildrenChanged(String path) {
594 if(path.equals(watcher.splitLogZNode)) {
595 LOG.debug("tasks arrived or departed");
596 synchronized (taskReadyLock) {
597 taskReadySeq++;
598 taskReadyLock.notify();
599 }
600 }
601 }
602
603
604
605
606
607 void stopTask() {
608 LOG.info("Sending interrupt to stop the worker thread");
609 worker.interrupt();
610 }
611
612
613
614
615
616 public void start() {
617 worker = new Thread(null, this, "SplitLogWorker-" + serverName);
618 exitWorker = false;
619 worker.start();
620 }
621
622
623
624
625 public void stop() {
626 exitWorker = true;
627 stopTask();
628 }
629
630
631
632
633 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
634 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
635
636 @Override
637 public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
638 SplitLogCounters.tot_wkr_get_data_result.incrementAndGet();
639 if (rc != 0) {
640 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
641 getDataSetWatchFailure(path);
642 return;
643 }
644 data = watcher.getRecoverableZooKeeper().removeMetaData(data);
645 getDataSetWatchSuccess(path, data);
646 }
647 }
648
649
650
651
652
653
654
655
656 public interface TaskExecutor {
657 enum Status {
658 DONE(),
659 ERR(),
660 RESIGNED(),
661 PREEMPTED()
662 }
663 Status exec(String name, CancelableProgressable p);
664 }
665 }