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