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.master;
21
22 import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
23
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Collections;
27 import java.util.HashSet;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.Set;
31 import java.util.concurrent.ConcurrentHashMap;
32 import java.util.concurrent.ConcurrentMap;
33
34 import org.apache.commons.logging.Log;
35 import org.apache.commons.logging.LogFactory;
36 import org.apache.hadoop.conf.Configuration;
37 import org.apache.hadoop.fs.FileStatus;
38 import org.apache.hadoop.fs.FileSystem;
39 import org.apache.hadoop.fs.Path;
40 import org.apache.hadoop.hbase.Chore;
41 import org.apache.hadoop.hbase.ServerName;
42 import org.apache.hadoop.hbase.Stoppable;
43 import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
44 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
45 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
46 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
47 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
48 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
49 import org.apache.hadoop.hbase.util.FSUtils;
50 import org.apache.hadoop.hbase.util.Threads;
51 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
52 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
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.CreateMode;
59 import org.apache.zookeeper.KeeperException;
60 import org.apache.zookeeper.KeeperException.NoNodeException;
61 import org.apache.zookeeper.ZooDefs.Ids;
62 import org.apache.zookeeper.data.Stat;
63
64 import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.*;
65 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*;
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96 public class SplitLogManager extends ZooKeeperListener {
97 private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
98
99 private final Stoppable stopper;
100 private final String serverName;
101 private final TaskFinisher taskFinisher;
102 private FileSystem fs;
103 private Configuration conf;
104
105 private long zkretries;
106 private long resubmit_threshold;
107 private long timeout;
108 private long unassignedTimeout;
109 private long lastNodeCreateTime = Long.MAX_VALUE;
110 public boolean ignoreZKDeleteForTesting = false;
111
112 private ConcurrentMap<String, Task> tasks =
113 new ConcurrentHashMap<String, Task>();
114 private TimeoutMonitor timeoutMonitor;
115
116 private Set<String> deadWorkers = null;
117 private Object deadWorkersLock = new Object();
118
119 private Set<String> failedDeletions = null;
120
121
122
123
124
125
126
127
128
129
130
131
132
133 public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
134 Stoppable stopper, String serverName) {
135 this(zkw, conf, stopper, serverName, new TaskFinisher() {
136 @Override
137 public Status finish(String workerName, String logfile) {
138 try {
139 HLogSplitter.finishSplitLogFile(logfile, conf);
140 } catch (IOException e) {
141 LOG.warn("Could not finish splitting of log file " + logfile, e);
142 return Status.ERR;
143 }
144 return Status.DONE;
145 }
146 });
147 }
148
149
150
151
152
153
154
155
156
157
158
159
160 public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
161 Stoppable stopper, String serverName, TaskFinisher tf) {
162 super(zkw);
163 this.taskFinisher = tf;
164 this.conf = conf;
165 this.stopper = stopper;
166 this.zkretries = conf.getLong("hbase.splitlog.zk.retries",
167 ZKSplitLog.DEFAULT_ZK_RETRIES);
168 this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit",
169 ZKSplitLog.DEFAULT_MAX_RESUBMIT);
170 this.timeout = conf.getInt("hbase.splitlog.manager.timeout",
171 ZKSplitLog.DEFAULT_TIMEOUT);
172 this.unassignedTimeout =
173 conf.getInt("hbase.splitlog.manager.unassigned.timeout",
174 ZKSplitLog.DEFAULT_UNASSIGNED_TIMEOUT);
175 LOG.debug("timeout = " + timeout);
176 LOG.debug("unassigned timeout = " + unassignedTimeout);
177
178 this.serverName = serverName;
179 this.timeoutMonitor = new TimeoutMonitor(
180 conf.getInt("hbase.splitlog.manager.timeoutmonitor.period",
181 1000),
182 stopper);
183
184 this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
185 }
186
187 public void finishInitialization(boolean masterRecovery) {
188 if (!masterRecovery) {
189 Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
190 + ".splitLogManagerTimeoutMonitor");
191 }
192
193 if (this.watcher != null) {
194 this.watcher.registerListener(this);
195 lookForOrphans();
196 }
197 }
198
199 private FileStatus[] getFileList(List<Path> logDirs) throws IOException {
200 List<FileStatus> fileStatus = new ArrayList<FileStatus>();
201 for (Path hLogDir : logDirs) {
202 this.fs = hLogDir.getFileSystem(conf);
203 if (!fs.exists(hLogDir)) {
204 LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
205 continue;
206 }
207
208 FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, null);
209 if (logfiles == null || logfiles.length == 0) {
210 LOG.info(hLogDir + " is empty dir, no logs to split");
211 } else {
212 for (FileStatus status : logfiles)
213 fileStatus.add(status);
214 }
215 }
216 FileStatus[] a = new FileStatus[fileStatus.size()];
217 return fileStatus.toArray(a);
218 }
219
220
221
222
223
224
225
226
227
228 public long splitLogDistributed(final Path logDir) throws IOException {
229 List<Path> logDirs = new ArrayList<Path>();
230 logDirs.add(logDir);
231 return splitLogDistributed(logDirs);
232 }
233
234
235
236
237
238
239
240
241
242
243
244 public long splitLogDistributed(final List<Path> logDirs) throws IOException {
245 MonitoredTask status = TaskMonitor.get().createStatus(
246 "Doing distributed log split in " + logDirs);
247 FileStatus[] logfiles = getFileList(logDirs);
248 status.setStatus("Checking directory contents...");
249 LOG.debug("Scheduling batch of logs to split");
250 tot_mgr_log_split_batch_start.incrementAndGet();
251 LOG.info("started splitting logs in " + logDirs);
252 long t = EnvironmentEdgeManager.currentTimeMillis();
253 long totalSize = 0;
254 TaskBatch batch = new TaskBatch();
255 for (FileStatus lf : logfiles) {
256
257
258
259
260
261 totalSize += lf.getLen();
262 if (enqueueSplitTask(lf.getPath().toString(), batch) == false) {
263 throw new IOException("duplicate log split scheduled for "
264 + lf.getPath());
265 }
266 }
267 waitForSplittingCompletion(batch, status);
268 if (batch.done != batch.installed) {
269 batch.isDead = true;
270 tot_mgr_log_split_batch_err.incrementAndGet();
271 LOG.warn("error while splitting logs in " + logDirs +
272 " installed = " + batch.installed + " but only " + batch.done + " done");
273 String msg = "error or interrupted while splitting logs in "
274 + logDirs + " Task = " + batch;
275 status.abort(msg);
276 throw new IOException(msg);
277 }
278 for(Path logDir: logDirs){
279 status.setStatus("Cleaning up log directory...");
280 try {
281 if (fs.exists(logDir) && !fs.delete(logDir, false)) {
282 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
283 }
284 } catch (IOException ioe) {
285 FileStatus[] files = fs.listStatus(logDir);
286 if (files != null && files.length > 0) {
287 LOG.warn("returning success without actually splitting and " +
288 "deleting all the log files in path " + logDir);
289 } else {
290 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
291 }
292 }
293 tot_mgr_log_split_batch_success.incrementAndGet();
294 }
295 String msg = "finished splitting (more than or equal to) " + totalSize +
296 " bytes in " + batch.installed + " log files in " + logDirs + " in " +
297 (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms";
298 status.markComplete(msg);
299 LOG.info(msg);
300 return totalSize;
301 }
302
303
304
305
306
307
308
309
310 boolean enqueueSplitTask(String taskname, TaskBatch batch) {
311 tot_mgr_log_split_start.incrementAndGet();
312 String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
313 Task oldtask = createTaskIfAbsent(path, batch);
314 if (oldtask == null) {
315
316 createNode(path, zkretries);
317 return true;
318 }
319 return false;
320 }
321
322 private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {
323 synchronized (batch) {
324 while ((batch.done + batch.error) != batch.installed) {
325 try {
326 status.setStatus("Waiting for distributed tasks to finish. "
327 + " scheduled=" + batch.installed
328 + " done=" + batch.done
329 + " error=" + batch.error);
330 int remaining = batch.installed - (batch.done + batch.error);
331 int actual = activeTasks(batch);
332 if (remaining != actual) {
333 LOG.warn("Expected " + remaining
334 + " active tasks, but actually there are " + actual);
335 }
336 int remainingInZK = remainingTasksInZK();
337 if (remainingInZK >= 0 && actual > remainingInZK) {
338 LOG.warn("Expected at least" + actual
339 + " tasks in ZK, but actually there are " + remainingInZK);
340 }
341 if (remainingInZK == 0 || actual == 0) {
342 LOG.warn("No more task remaining (ZK or task map), splitting "
343 + "should have completed. Remaining tasks in ZK " + remainingInZK
344 + ", active tasks in map " + actual);
345 if (remainingInZK == 0 && actual == 0) {
346 return;
347 }
348 }
349 batch.wait(100);
350 if (stopper.isStopped()) {
351 LOG.warn("Stopped while waiting for log splits to be completed");
352 return;
353 }
354 } catch (InterruptedException e) {
355 LOG.warn("Interrupted while waiting for log splits to be completed");
356 Thread.currentThread().interrupt();
357 return;
358 }
359 }
360 }
361 }
362
363 private int activeTasks(final TaskBatch batch) {
364 int count = 0;
365 for (Task t: tasks.values()) {
366 if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
367 count++;
368 }
369 }
370 return count;
371 }
372
373 private int remainingTasksInZK() {
374 int count = 0;
375 try {
376 List<String> tasks =
377 ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
378 if (tasks != null) {
379 for (String t: tasks) {
380 if (!ZKSplitLog.isRescanNode(watcher, t)) {
381 count++;
382 }
383 }
384 }
385 } catch (KeeperException ke) {
386 LOG.warn("Failed to check remaining tasks", ke);
387 count = -1;
388 }
389 return count;
390 }
391
392 private void setDone(String path, TerminationStatus status) {
393 Task task = tasks.get(path);
394 if (task == null) {
395 if (!ZKSplitLog.isRescanNode(watcher, path)) {
396 tot_mgr_unacquired_orphan_done.incrementAndGet();
397 LOG.debug("unacquired orphan task is done " + path);
398 }
399 } else {
400 synchronized (task) {
401 if (task.status == IN_PROGRESS) {
402 if (status == SUCCESS) {
403 tot_mgr_log_split_success.incrementAndGet();
404 LOG.info("Done splitting " + path);
405 } else {
406 tot_mgr_log_split_err.incrementAndGet();
407 LOG.warn("Error splitting " + path);
408 }
409 task.status = status;
410 if (task.batch != null) {
411 synchronized (task.batch) {
412 if (status == SUCCESS) {
413 task.batch.done++;
414 } else {
415 task.batch.error++;
416 }
417 task.batch.notify();
418 }
419 }
420 }
421 }
422 }
423
424
425
426
427
428 deleteNode(path, zkretries);
429 return;
430 }
431
432 private void createNode(String path, Long retry_count) {
433 ZKUtil.asyncCreate(this.watcher, path,
434 TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(),
435 retry_count);
436 tot_mgr_node_create_queued.incrementAndGet();
437 return;
438 }
439
440 private void createNodeSuccess(String path) {
441 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
442 LOG.debug("put up splitlog task at znode " + path);
443 getDataSetWatch(path, zkretries);
444 }
445
446 private void createNodeFailure(String path) {
447
448 LOG.warn("failed to create task node" + path);
449 setDone(path, FAILURE);
450 }
451
452
453 private void getDataSetWatch(String path, Long retry_count) {
454 this.watcher.getRecoverableZooKeeper().getZooKeeper().
455 getData(path, this.watcher,
456 new GetDataAsyncCallback(), retry_count);
457 tot_mgr_get_data_queued.incrementAndGet();
458 }
459
460 private void tryGetDataSetWatch(String path) {
461
462 this.watcher.getRecoverableZooKeeper().getZooKeeper().
463 getData(path, this.watcher,
464 new GetDataAsyncCallback(), new Long(-1)
465 tot_mgr_get_data_queued.incrementAndGet();
466 }
467
468 private void getDataSetWatchSuccess(String path, byte[] data, int version) {
469 if (data == null) {
470 if (version == Integer.MIN_VALUE) {
471
472 setDone(path, SUCCESS);
473 return;
474 }
475 tot_mgr_null_data.incrementAndGet();
476 LOG.fatal("logic error - got null data " + path);
477 setDone(path, FAILURE);
478 return;
479 }
480 data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
481
482 if (TaskState.TASK_UNASSIGNED.equals(data)) {
483 LOG.debug("task not yet acquired " + path + " ver = " + version);
484 handleUnassignedTask(path);
485 } else if (TaskState.TASK_OWNED.equals(data)) {
486 heartbeat(path, version,
487 TaskState.TASK_OWNED.getWriterName(data));
488 } else if (TaskState.TASK_RESIGNED.equals(data)) {
489 LOG.info("task " + path + " entered state " + new String(data));
490 resubmitOrFail(path, FORCE);
491 } else if (TaskState.TASK_DONE.equals(data)) {
492 LOG.info("task " + path + " entered state " + new String(data));
493 if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
494 if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data),
495 ZKSplitLog.getFileName(path)) == Status.DONE) {
496 setDone(path, SUCCESS);
497 } else {
498 resubmitOrFail(path, CHECK);
499 }
500 } else {
501 setDone(path, SUCCESS);
502 }
503 } else if (TaskState.TASK_ERR.equals(data)) {
504 LOG.info("task " + path + " entered state " + new String(data));
505 resubmitOrFail(path, CHECK);
506 } else {
507 LOG.fatal("logic error - unexpected zk state for path = " + path
508 + " data = " + new String(data));
509 setDone(path, FAILURE);
510 }
511 }
512
513 private void getDataSetWatchFailure(String path) {
514 LOG.warn("failed to set data watch " + path);
515 setDone(path, FAILURE);
516 }
517
518
519
520
521
522
523
524
525
526
527 private void handleUnassignedTask(String path) {
528 if (ZKSplitLog.isRescanNode(watcher, path)) {
529 return;
530 }
531 Task task = findOrCreateOrphanTask(path);
532 if (task.isOrphan() && (task.incarnation == 0)) {
533 LOG.info("resubmitting unassigned orphan task " + path);
534
535
536 resubmit(path, task, FORCE);
537 }
538 }
539
540
541
542
543
544
545
546 private boolean shouldAbandonRetries(int statusCode, String action) {
547 if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
548 LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
549 + "action=" + action);
550 return true;
551 }
552 return false;
553 }
554
555 private void heartbeat(String path, int new_version,
556 String workerName) {
557 Task task = findOrCreateOrphanTask(path);
558 if (new_version != task.last_version) {
559 if (task.isUnassigned()) {
560 LOG.info("task " + path + " acquired by " + workerName);
561 }
562 task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(),
563 new_version, workerName);
564 tot_mgr_heartbeat.incrementAndGet();
565 } else {
566
567
568
569
570 }
571 return;
572 }
573
574 private boolean resubmit(String path, Task task,
575 ResubmitDirective directive) {
576
577
578 if (task.status != IN_PROGRESS) {
579 return false;
580 }
581 int version;
582 if (directive != FORCE) {
583 if ((EnvironmentEdgeManager.currentTimeMillis() - task.last_update) <
584 timeout) {
585 return false;
586 }
587 if (task.unforcedResubmits >= resubmit_threshold) {
588 if (!task.resubmitThresholdReached) {
589 task.resubmitThresholdReached = true;
590 tot_mgr_resubmit_threshold_reached.incrementAndGet();
591 LOG.info("Skipping resubmissions of task " + path +
592 " because threshold " + resubmit_threshold + " reached");
593 }
594 return false;
595 }
596
597 version = task.last_version;
598 } else {
599 version = -1;
600 }
601 LOG.info("resubmitting task " + path);
602 task.incarnation++;
603 try {
604
605 if (ZKUtil.setData(this.watcher, path,
606 TaskState.TASK_UNASSIGNED.get(serverName),
607 version) == false) {
608 LOG.debug("failed to resubmit task " + path +
609 " version changed");
610 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
611 return false;
612 }
613 } catch (NoNodeException e) {
614 LOG.warn("failed to resubmit because znode doesn't exist " + path +
615 " task done (or forced done by removing the znode)");
616 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
617 return false;
618 } catch (KeeperException.BadVersionException e) {
619 LOG.debug("failed to resubmit task " + path +
620 " version changed");
621 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
622 return false;
623 } catch (KeeperException e) {
624 tot_mgr_resubmit_failed.incrementAndGet();
625 LOG.warn("failed to resubmit " + path, e);
626 return false;
627 }
628
629 if (directive != FORCE) {
630 task.unforcedResubmits++;
631 }
632 task.setUnassigned();
633 createRescanNode(Long.MAX_VALUE);
634 tot_mgr_resubmit.incrementAndGet();
635 return true;
636 }
637
638 private void resubmitOrFail(String path, ResubmitDirective directive) {
639 if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
640 setDone(path, FAILURE);
641 }
642 }
643
644 private void deleteNode(String path, Long retries) {
645 tot_mgr_node_delete_queued.incrementAndGet();
646
647
648
649 this.watcher.getRecoverableZooKeeper().getZooKeeper().
650 delete(path, -1, new DeleteAsyncCallback(),
651 retries);
652 }
653
654 private void deleteNodeSuccess(String path) {
655 if (ignoreZKDeleteForTesting) {
656 return;
657 }
658 Task task;
659 task = tasks.remove(path);
660 if (task == null) {
661 if (ZKSplitLog.isRescanNode(watcher, path)) {
662 tot_mgr_rescan_deleted.incrementAndGet();
663 }
664 tot_mgr_missing_state_in_delete.incrementAndGet();
665 LOG.debug("deleted task without in memory state " + path);
666 return;
667 }
668 synchronized (task) {
669 task.status = DELETED;
670 task.notify();
671 }
672 tot_mgr_task_deleted.incrementAndGet();
673 }
674
675 private void deleteNodeFailure(String path) {
676 LOG.info("Failed to delete node " + path + " and will retry soon.");
677 return;
678 }
679
680
681
682
683
684
685 private void createRescanNode(long retries) {
686
687
688
689
690
691
692
693 this.watcher.getRecoverableZooKeeper().getZooKeeper().
694 create(ZKSplitLog.getRescanNode(watcher),
695 TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE,
696 CreateMode.EPHEMERAL_SEQUENTIAL,
697 new CreateRescanAsyncCallback(), Long.valueOf(retries));
698 }
699
700 private void createRescanSuccess(String path) {
701 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
702 tot_mgr_rescan.incrementAndGet();
703 getDataSetWatch(path, zkretries);
704 }
705
706 private void createRescanFailure() {
707 LOG.fatal("logic failure, rescan failure must not happen");
708 }
709
710
711
712
713
714
715 private Task createTaskIfAbsent(String path, TaskBatch batch) {
716 Task oldtask;
717
718
719 Task newtask = new Task();
720 newtask.batch = batch;
721 oldtask = tasks.putIfAbsent(path, newtask);
722 if (oldtask == null) {
723 batch.installed++;
724 return null;
725 }
726
727 synchronized (oldtask) {
728 if (oldtask.isOrphan()) {
729 if (oldtask.status == SUCCESS) {
730
731
732
733
734 return (null);
735 }
736 if (oldtask.status == IN_PROGRESS) {
737 oldtask.batch = batch;
738 batch.installed++;
739 LOG.debug("Previously orphan task " + path +
740 " is now being waited upon");
741 return null;
742 }
743 while (oldtask.status == FAILURE) {
744 LOG.debug("wait for status of task " + path +
745 " to change to DELETED");
746 tot_mgr_wait_for_zk_delete.incrementAndGet();
747 try {
748 oldtask.wait();
749 } catch (InterruptedException e) {
750 Thread.currentThread().interrupt();
751 LOG.warn("Interrupted when waiting for znode delete callback");
752
753 break;
754 }
755 }
756 if (oldtask.status != DELETED) {
757 LOG.warn("Failure because previously failed task" +
758 " state still present. Waiting for znode delete callback" +
759 " path=" + path);
760 return oldtask;
761 }
762
763 Task t = tasks.putIfAbsent(path, newtask);
764 if (t == null) {
765 batch.installed++;
766 return null;
767 }
768 LOG.fatal("Logic error. Deleted task still present in tasks map");
769 assert false : "Deleted task still present in tasks map";
770 return t;
771 }
772 LOG.warn("Failure because two threads can't wait for the same task. " +
773 " path=" + path);
774 return oldtask;
775 }
776 }
777
778 Task findOrCreateOrphanTask(String path) {
779 Task orphanTask = new Task();
780 Task task;
781 task = tasks.putIfAbsent(path, orphanTask);
782 if (task == null) {
783 LOG.info("creating orphan task " + path);
784 tot_mgr_orphan_task_acquired.incrementAndGet();
785 task = orphanTask;
786 }
787 return task;
788 }
789
790 @Override
791 public void nodeDataChanged(String path) {
792 Task task;
793 task = tasks.get(path);
794 if (task != null || ZKSplitLog.isRescanNode(watcher, path)) {
795 if (task != null) {
796 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
797 }
798 getDataSetWatch(path, zkretries);
799 }
800 }
801
802 public void stop() {
803 if (timeoutMonitor != null) {
804 timeoutMonitor.interrupt();
805 }
806 }
807
808 private void lookForOrphans() {
809 List<String> orphans;
810 try {
811 orphans = ZKUtil.listChildrenNoWatch(this.watcher,
812 this.watcher.splitLogZNode);
813 if (orphans == null) {
814 LOG.warn("could not get children of " + this.watcher.splitLogZNode);
815 return;
816 }
817 } catch (KeeperException e) {
818 LOG.warn("could not get children of " + this.watcher.splitLogZNode +
819 " " + StringUtils.stringifyException(e));
820 return;
821 }
822 int rescan_nodes = 0;
823 for (String path : orphans) {
824 String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
825 if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
826 rescan_nodes++;
827 LOG.debug("found orphan rescan node " + path);
828 } else {
829 LOG.info("found orphan task " + path);
830 }
831 getDataSetWatch(nodepath, zkretries);
832 }
833 LOG.info("found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
834 rescan_nodes + " rescan nodes");
835 }
836
837
838
839
840
841
842
843
844 static class TaskBatch {
845 int installed = 0;
846 int done = 0;
847 int error = 0;
848 volatile boolean isDead = false;
849
850 @Override
851 public String toString() {
852 return ("installed = " + installed + " done = " + done + " error = "
853 + error);
854 }
855 }
856
857
858
859
860 static class Task {
861 volatile long last_update;
862 volatile int last_version;
863 volatile String cur_worker_name;
864 TaskBatch batch;
865 volatile TerminationStatus status;
866 volatile int incarnation;
867 volatile int unforcedResubmits;
868 volatile boolean resubmitThresholdReached;
869
870 @Override
871 public String toString() {
872 return ("last_update = " + last_update +
873 " last_version = " + last_version +
874 " cur_worker_name = " + cur_worker_name +
875 " status = " + status +
876 " incarnation = " + incarnation +
877 " resubmits = " + unforcedResubmits +
878 " batch = " + batch);
879 }
880
881 Task() {
882 incarnation = 0;
883 last_version = -1;
884 status = IN_PROGRESS;
885 setUnassigned();
886 }
887
888 public boolean isOrphan() {
889 return (batch == null || batch.isDead);
890 }
891
892 public boolean isUnassigned() {
893 return (cur_worker_name == null);
894 }
895
896 public void heartbeatNoDetails(long time) {
897 last_update = time;
898 }
899
900 public void heartbeat(long time, int version, String worker) {
901 last_version = version;
902 last_update = time;
903 cur_worker_name = worker;
904 }
905
906 public void setUnassigned() {
907 cur_worker_name = null;
908 last_update = -1;
909 }
910 }
911
912 void handleDeadWorker(String workerName) {
913
914
915 synchronized (deadWorkersLock) {
916 if (deadWorkers == null) {
917 deadWorkers = new HashSet<String>(100);
918 }
919 deadWorkers.add(workerName);
920 }
921 LOG.info("dead splitlog worker " + workerName);
922 }
923
924 void handleDeadWorkers(List<ServerName> serverNames) {
925 List<String> workerNames = new ArrayList<String>(serverNames.size());
926 for (ServerName serverName : serverNames) {
927 workerNames.add(serverName.toString());
928 }
929 synchronized (deadWorkersLock) {
930 if (deadWorkers == null) {
931 deadWorkers = new HashSet<String>(100);
932 }
933 deadWorkers.addAll(workerNames);
934 }
935 LOG.info("dead splitlog workers " + workerNames);
936 }
937
938
939
940
941
942 private class TimeoutMonitor extends Chore {
943 public TimeoutMonitor(final int period, Stoppable stopper) {
944 super("SplitLogManager Timeout Monitor", period, stopper);
945 }
946
947 @Override
948 protected void chore() {
949 int resubmitted = 0;
950 int unassigned = 0;
951 int tot = 0;
952 boolean found_assigned_task = false;
953 Set<String> localDeadWorkers;
954
955 synchronized (deadWorkersLock) {
956 localDeadWorkers = deadWorkers;
957 deadWorkers = null;
958 }
959
960 for (Map.Entry<String, Task> e : tasks.entrySet()) {
961 String path = e.getKey();
962 Task task = e.getValue();
963 String cur_worker = task.cur_worker_name;
964 tot++;
965
966
967
968
969
970 if (task.isUnassigned()) {
971 unassigned++;
972 continue;
973 }
974 found_assigned_task = true;
975 if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
976 tot_mgr_resubmit_dead_server_task.incrementAndGet();
977 if (resubmit(path, task, FORCE)) {
978 resubmitted++;
979 } else {
980 handleDeadWorker(cur_worker);
981 LOG.warn("Failed to resubmit task " + path + " owned by dead " +
982 cur_worker + ", will retry.");
983 }
984 } else if (resubmit(path, task, CHECK)) {
985 resubmitted++;
986 }
987 }
988 if (tot > 0) {
989 LOG.debug("total tasks = " + tot + " unassigned = " + unassigned);
990 }
991 if (resubmitted > 0) {
992 LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
993 }
994
995
996
997
998
999
1000
1001
1002
1003 if (tot > 0 && !found_assigned_task &&
1004 ((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
1005 unassignedTimeout)) {
1006 for (Map.Entry<String, Task> e : tasks.entrySet()) {
1007 String path = e.getKey();
1008 Task task = e.getValue();
1009
1010
1011
1012
1013 if (task.isUnassigned() && (task.status != FAILURE)) {
1014
1015 tryGetDataSetWatch(path);
1016 }
1017 }
1018 createRescanNode(Long.MAX_VALUE);
1019 tot_mgr_resubmit_unassigned.incrementAndGet();
1020 LOG.debug("resubmitting unassigned task(s) after timeout");
1021 }
1022
1023
1024 if (failedDeletions.size() > 0) {
1025 List<String> tmpPaths = new ArrayList<String>(failedDeletions);
1026 failedDeletions.removeAll(tmpPaths);
1027 for (String tmpPath : tmpPaths) {
1028
1029 deleteNode(tmpPath, zkretries);
1030 }
1031 }
1032 }
1033 }
1034
1035
1036
1037
1038
1039 class CreateAsyncCallback implements AsyncCallback.StringCallback {
1040 private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
1041
1042 @Override
1043 public void processResult(int rc, String path, Object ctx, String name) {
1044 tot_mgr_node_create_result.incrementAndGet();
1045 if (rc != 0) {
1046 if (shouldAbandonRetries(rc, "Create znode " + path)) {
1047 createNodeFailure(path);
1048 return;
1049 }
1050 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
1051
1052
1053
1054
1055
1056
1057 LOG.debug("found pre-existing znode " + path);
1058 tot_mgr_node_already_exists.incrementAndGet();
1059 } else {
1060 Long retry_count = (Long)ctx;
1061 LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
1062 path + " remaining retries=" + retry_count);
1063 if (retry_count == 0) {
1064 tot_mgr_node_create_err.incrementAndGet();
1065 createNodeFailure(path);
1066 } else {
1067 tot_mgr_node_create_retry.incrementAndGet();
1068 createNode(path, retry_count - 1);
1069 }
1070 return;
1071 }
1072 }
1073 createNodeSuccess(path);
1074 }
1075 }
1076
1077
1078
1079
1080
1081 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
1082 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
1083
1084 @Override
1085 public void processResult(int rc, String path, Object ctx, byte[] data,
1086 Stat stat) {
1087 tot_mgr_get_data_result.incrementAndGet();
1088 if (rc != 0) {
1089 if (shouldAbandonRetries(rc, "GetData from znode " + path)) {
1090 return;
1091 }
1092 if (rc == KeeperException.Code.NONODE.intValue()) {
1093 tot_mgr_get_data_nonode.incrementAndGet();
1094
1095
1096
1097 LOG.warn("task znode " + path + " vanished.");
1098 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
1099 return;
1100 }
1101 Long retry_count = (Long) ctx;
1102
1103 if (retry_count < 0) {
1104 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1105 path + ". Ignoring error. No error handling. No retrying.");
1106 return;
1107 }
1108 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1109 path + " remaining retries=" + retry_count);
1110 if (retry_count == 0) {
1111 tot_mgr_get_data_err.incrementAndGet();
1112 getDataSetWatchFailure(path);
1113 } else {
1114 tot_mgr_get_data_retry.incrementAndGet();
1115 getDataSetWatch(path, retry_count - 1);
1116 }
1117 return;
1118 }
1119 getDataSetWatchSuccess(path, data, stat.getVersion());
1120 return;
1121 }
1122 }
1123
1124
1125
1126
1127
1128 class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
1129 private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
1130
1131 @Override
1132 public void processResult(int rc, String path, Object ctx) {
1133 tot_mgr_node_delete_result.incrementAndGet();
1134 if (rc != 0) {
1135 if (shouldAbandonRetries(rc, "Delete znode " + path)) {
1136 failedDeletions.add(path);
1137 return;
1138 }
1139 if (rc != KeeperException.Code.NONODE.intValue()) {
1140 tot_mgr_node_delete_err.incrementAndGet();
1141 Long retry_count = (Long) ctx;
1142 LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
1143 path + " remaining retries=" + retry_count);
1144 if (retry_count == 0) {
1145 LOG.warn("delete failed " + path);
1146 failedDeletions.add(path);
1147 deleteNodeFailure(path);
1148 } else {
1149 deleteNode(path, retry_count - 1);
1150 }
1151 return;
1152 } else {
1153 LOG.debug(path +
1154 " does not exist. Either was created but deleted behind our" +
1155 " back by another pending delete OR was deleted" +
1156 " in earlier retry rounds. zkretries = " + (Long) ctx);
1157 }
1158 } else {
1159 LOG.debug("deleted " + path);
1160 }
1161 deleteNodeSuccess(path);
1162 }
1163 }
1164
1165
1166
1167
1168
1169
1170
1171
1172 class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
1173 private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
1174
1175 @Override
1176 public void processResult(int rc, String path, Object ctx, String name) {
1177 if (rc != 0) {
1178 if (shouldAbandonRetries(rc, "CreateRescan znode " + path)) {
1179 return;
1180 }
1181 Long retry_count = (Long)ctx;
1182 LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
1183 " remaining retries=" + retry_count);
1184 if (retry_count == 0) {
1185 createRescanFailure();
1186 } else {
1187 createRescanNode(retry_count - 1);
1188 }
1189 return;
1190 }
1191
1192 createRescanSuccess(name);
1193 }
1194 }
1195
1196
1197
1198
1199
1200
1201
1202 static public interface TaskFinisher {
1203
1204
1205
1206 static public enum Status {
1207
1208
1209
1210 DONE(),
1211
1212
1213
1214 ERR();
1215 }
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225 public Status finish(String workerName, String taskname);
1226 }
1227 enum ResubmitDirective {
1228 CHECK(),
1229 FORCE();
1230 }
1231 enum TerminationStatus {
1232 IN_PROGRESS("in_progress"),
1233 SUCCESS("success"),
1234 FAILURE("failure"),
1235 DELETED("deleted");
1236
1237 String statusMsg;
1238 TerminationStatus(String msg) {
1239 statusMsg = msg;
1240 }
1241
1242 @Override
1243 public String toString() {
1244 return statusMsg;
1245 }
1246 }
1247
1248
1249
1250
1251 public void finishInitialization() {
1252 finishInitialization(false);
1253 }
1254 }