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