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