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.HashSet;
27 import java.util.Set;
28 import java.util.List;
29 import java.util.Map;
30 import java.util.concurrent.ConcurrentHashMap;
31 import java.util.concurrent.ConcurrentMap;
32
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.conf.Configuration;
36 import org.apache.hadoop.fs.FileStatus;
37 import org.apache.hadoop.fs.FileSystem;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.Chore;
40 import org.apache.hadoop.hbase.Stoppable;
41 import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
42 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
43 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
44 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
45 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
46 import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
47 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
48 import org.apache.hadoop.hbase.util.FSUtils;
49 import org.apache.hadoop.hbase.util.Threads;
50 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
51 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
52 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
53 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
54 import org.apache.hadoop.util.StringUtils;
55 import org.apache.zookeeper.AsyncCallback;
56 import org.apache.zookeeper.CreateMode;
57 import org.apache.zookeeper.KeeperException;
58 import org.apache.zookeeper.KeeperException.NoNodeException;
59 import org.apache.zookeeper.ZooDefs.Ids;
60 import org.apache.zookeeper.data.Stat;
61 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
62
63 import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.*;
64 import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*;
65
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 public class SplitLogManager extends ZooKeeperListener {
96 private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
97
98 private final Stoppable stopper;
99 private final String serverName;
100 private final TaskFinisher taskFinisher;
101 private FileSystem fs;
102 private Configuration conf;
103
104 private long zkretries;
105 private long resubmit_threshold;
106 private long timeout;
107 private long unassignedTimeout;
108 private long lastNodeCreateTime = Long.MAX_VALUE;
109
110 private ConcurrentMap<String, Task> tasks =
111 new ConcurrentHashMap<String, Task>();
112 private TimeoutMonitor timeoutMonitor;
113
114 private Set<String> deadWorkers = null;
115 private Object deadWorkersLock = new Object();
116
117
118
119
120
121
122
123
124
125
126
127 public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
128 Stoppable stopper, String serverName) {
129 this(zkw, conf, stopper, serverName, new TaskFinisher() {
130 @Override
131 public Status finish(String workerName, String logfile) {
132 String tmpname =
133 ZKSplitLog.getSplitLogDirTmpComponent(workerName, logfile);
134 try {
135 HLogSplitter.moveRecoveredEditsFromTemp(tmpname, logfile, conf);
136 } catch (IOException e) {
137 LOG.warn("Could not finish splitting of log file " + logfile);
138 return Status.ERR;
139 }
140 return Status.DONE;
141 }
142 });
143 }
144 public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
145 Stoppable stopper, String serverName, TaskFinisher tf) {
146 super(zkw);
147 this.taskFinisher = tf;
148 this.conf = conf;
149 this.stopper = stopper;
150 this.zkretries = conf.getLong("hbase.splitlog.zk.retries",
151 ZKSplitLog.DEFAULT_ZK_RETRIES);
152 this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit",
153 ZKSplitLog.DEFAULT_MAX_RESUBMIT);
154 this.timeout = conf.getInt("hbase.splitlog.manager.timeout",
155 ZKSplitLog.DEFAULT_TIMEOUT);
156 this.unassignedTimeout =
157 conf.getInt("hbase.splitlog.manager.unassigned.timeout",
158 ZKSplitLog.DEFAULT_UNASSIGNED_TIMEOUT);
159 LOG.debug("timeout = " + timeout);
160 LOG.debug("unassigned timeout = " + unassignedTimeout);
161
162 this.serverName = serverName;
163 this.timeoutMonitor = new TimeoutMonitor(
164 conf.getInt("hbase.splitlog.manager.timeoutmonitor.period",
165 1000),
166 stopper);
167 }
168
169 public void finishInitialization() {
170 Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName +
171 ".splitLogManagerTimeoutMonitor");
172
173 if (this.watcher != null) {
174 this.watcher.registerListener(this);
175 lookForOrphans();
176 }
177 }
178
179 private FileStatus[] getFileList(List<Path> logDirs) throws IOException {
180 List<FileStatus> fileStatus = new ArrayList<FileStatus>();
181 for (Path hLogDir : logDirs) {
182 this.fs = hLogDir.getFileSystem(conf);
183 if (!fs.exists(hLogDir)) {
184 LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
185 continue;
186 }
187
188 FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, null);
189 if (logfiles == null || logfiles.length == 0) {
190 LOG.info(hLogDir + " is empty dir, no logs to split");
191 } else {
192 for (FileStatus status : logfiles)
193 fileStatus.add(status);
194 }
195 }
196 if (fileStatus.isEmpty())
197 return null;
198 FileStatus[] a = new FileStatus[fileStatus.size()];
199 return fileStatus.toArray(a);
200 }
201
202
203
204
205
206
207
208
209
210 public long splitLogDistributed(final Path logDir) throws IOException {
211 List<Path> logDirs = new ArrayList<Path>();
212 logDirs.add(logDir);
213 return splitLogDistributed(logDirs);
214 }
215
216
217
218
219
220
221
222
223
224
225
226 public long splitLogDistributed(final List<Path> logDirs) throws IOException {
227 MonitoredTask status = TaskMonitor.get().createStatus(
228 "Doing distributed log split in " + logDirs);
229 FileStatus[] logfiles = getFileList(logDirs);
230 if(logfiles == null)
231 return 0;
232 status.setStatus("Checking directory contents...");
233 LOG.debug("Scheduling batch of logs to split");
234 tot_mgr_log_split_batch_start.incrementAndGet();
235 LOG.info("started splitting logs in " + logDirs);
236 long t = EnvironmentEdgeManager.currentTimeMillis();
237 long totalSize = 0;
238 TaskBatch batch = new TaskBatch();
239 for (FileStatus lf : logfiles) {
240
241
242
243
244
245 totalSize += lf.getLen();
246 if (installTask(lf.getPath().toString(), batch) == false) {
247 throw new IOException("duplicate log split scheduled for "
248 + lf.getPath());
249 }
250 }
251 waitTasks(batch, status);
252 if (batch.done != batch.installed) {
253 stopTrackingTasks(batch);
254 tot_mgr_log_split_batch_err.incrementAndGet();
255 LOG.warn("error while splitting logs in " + logDirs +
256 " installed = " + batch.installed + " but only " + batch.done + " done");
257 throw new IOException("error or interrupt while splitting logs in "
258 + logDirs + " Task = " + batch);
259 }
260 for(Path logDir: logDirs){
261 if (anyNewLogFiles(logDir, logfiles)) {
262 tot_mgr_new_unexpected_hlogs.incrementAndGet();
263 LOG.warn("new hlogs were produced while logs in " + logDir +
264 " were being split");
265 throw new OrphanHLogAfterSplitException();
266 }
267 tot_mgr_log_split_batch_success.incrementAndGet();
268 status.setStatus("Cleaning up log directory...");
269 if (!fs.delete(logDir, true)) {
270 throw new IOException("Unable to delete src dir: " + logDir);
271 }
272 }
273 String msg = "finished splitting (more than or equal to) " + totalSize +
274 " bytes in " + batch.installed + " log files in " + logDirs + " in " +
275 (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms";
276 status.markComplete(msg);
277 LOG.info(msg);
278 return totalSize;
279 }
280
281 boolean installTask(String taskname, TaskBatch batch) {
282 tot_mgr_log_split_start.incrementAndGet();
283 String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
284 Task oldtask = createTaskIfAbsent(path, batch);
285 if (oldtask == null) {
286
287 createNode(path, zkretries);
288 return true;
289 }
290 LOG.warn(path + "is already being split. " +
291 "Two threads cannot wait for the same task");
292 return false;
293 }
294
295 private void waitTasks(TaskBatch batch, MonitoredTask status) {
296 synchronized (batch) {
297 while ((batch.done + batch.error) != batch.installed) {
298 try {
299 status.setStatus("Waiting for distributed tasks to finish. "
300 + " scheduled=" + batch.installed
301 + " done=" + batch.done
302 + " error=" + batch.error);
303 batch.wait(100);
304 if (stopper.isStopped()) {
305 LOG.warn("Stopped while waiting for log splits to be completed");
306 return;
307 }
308 } catch (InterruptedException e) {
309 LOG.warn("Interrupted while waiting for log splits to be completed");
310 Thread.currentThread().interrupt();
311 return;
312 }
313 }
314 }
315 }
316
317 private void setDone(String path, TerminationStatus status) {
318 if (!ZKSplitLog.isRescanNode(watcher, path)) {
319 if (status == SUCCESS) {
320 tot_mgr_log_split_success.incrementAndGet();
321 LOG.info("Done splitting " + path);
322 } else {
323 tot_mgr_log_split_err.incrementAndGet();
324 LOG.warn("Error splitting " + path);
325 }
326 }
327 Task task = tasks.get(path);
328 if (task == null) {
329 if (!ZKSplitLog.isRescanNode(watcher, path)) {
330 tot_mgr_unacquired_orphan_done.incrementAndGet();
331 LOG.debug("unacquired orphan task is done " + path);
332 }
333 } else {
334
335
336
337 if (!task.isOrphan()) {
338 synchronized (task.batch) {
339 if (status == SUCCESS) {
340 task.batch.done++;
341 } else {
342 task.batch.error++;
343 }
344 if ((task.batch.done + task.batch.error) == task.batch.installed) {
345 task.batch.notify();
346 }
347 }
348 }
349 task.deleted = true;
350 }
351
352
353
354
355 deleteNode(path, Long.MAX_VALUE);
356 return;
357 }
358
359 private void createNode(String path, Long retry_count) {
360 ZKUtil.asyncCreate(this.watcher, path,
361 TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(),
362 retry_count);
363 tot_mgr_node_create_queued.incrementAndGet();
364 return;
365 }
366
367 private void createNodeSuccess(String path) {
368 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
369 LOG.debug("put up splitlog task at znode " + path);
370 getDataSetWatch(path, zkretries);
371 }
372
373 private void createNodeFailure(String path) {
374
375 LOG.warn("failed to create task node" + path);
376 setDone(path, FAILURE);
377 }
378
379
380 private void getDataSetWatch(String path, Long retry_count) {
381 this.watcher.getRecoverableZooKeeper().getZooKeeper().
382 getData(path, this.watcher,
383 new GetDataAsyncCallback(), retry_count);
384 tot_mgr_get_data_queued.incrementAndGet();
385 }
386
387 private void getDataSetWatchSuccess(String path, byte[] data, int version) {
388 if (data == null) {
389 tot_mgr_null_data.incrementAndGet();
390 LOG.fatal("logic error - got null data " + path);
391 setDone(path, FAILURE);
392 return;
393 }
394 data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
395
396 if (TaskState.TASK_UNASSIGNED.equals(data)) {
397 LOG.debug("task not yet acquired " + path + " ver = " + version);
398 handleUnassignedTask(path);
399 } else if (TaskState.TASK_OWNED.equals(data)) {
400 heartbeat(path, version,
401 TaskState.TASK_OWNED.getWriterName(data));
402 } else if (TaskState.TASK_RESIGNED.equals(data)) {
403 LOG.info("task " + path + " entered state " + new String(data));
404 resubmitOrFail(path, FORCE);
405 } else if (TaskState.TASK_DONE.equals(data)) {
406 LOG.info("task " + path + " entered state " + new String(data));
407 if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
408 if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data),
409 ZKSplitLog.getFileName(path)) == Status.DONE) {
410 setDone(path, SUCCESS);
411 } else {
412 resubmitOrFail(path, CHECK);
413 }
414 } else {
415 setDone(path, SUCCESS);
416 }
417 } else if (TaskState.TASK_ERR.equals(data)) {
418 LOG.info("task " + path + " entered state " + new String(data));
419 resubmitOrFail(path, CHECK);
420 } else {
421 LOG.fatal("logic error - unexpected zk state for path = " + path
422 + " data = " + new String(data));
423 setDone(path, FAILURE);
424 }
425 }
426
427 private void getDataSetWatchFailure(String path) {
428 LOG.warn("failed to set data watch " + path);
429 setDone(path, FAILURE);
430 }
431
432
433
434
435
436
437
438
439
440
441 private void handleUnassignedTask(String path) {
442 if (ZKSplitLog.isRescanNode(watcher, path)) {
443 return;
444 }
445 Task task = findOrCreateOrphanTask(path);
446 if (task.isOrphan() && (task.incarnation == 0)) {
447 LOG.info("resubmitting unassigned orphan task " + path);
448
449
450 resubmit(path, task, FORCE);
451 }
452 }
453
454 private void heartbeat(String path, int new_version,
455 String workerName) {
456 Task task = findOrCreateOrphanTask(path);
457 if (new_version != task.last_version) {
458 if (task.isUnassigned()) {
459 LOG.info("task " + path + " acquired by " + workerName);
460 }
461 task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(),
462 new_version, workerName);
463 tot_mgr_heartbeat.incrementAndGet();
464 } else {
465 assert false;
466 LOG.warn("got dup heartbeat for " + path + " ver = " + new_version);
467 }
468 return;
469 }
470
471 private boolean resubmit(String path, Task task,
472 ResubmitDirective directive) {
473
474
475 if (task.deleted) {
476 return false;
477 }
478 int version;
479 if (directive != FORCE) {
480 if ((EnvironmentEdgeManager.currentTimeMillis() - task.last_update) <
481 timeout) {
482 return false;
483 }
484 if (task.unforcedResubmits >= resubmit_threshold) {
485 if (task.unforcedResubmits == resubmit_threshold) {
486 tot_mgr_resubmit_threshold_reached.incrementAndGet();
487 LOG.info("Skipping resubmissions of task " + path +
488 " because threshold " + resubmit_threshold + " reached");
489 }
490 return false;
491 }
492
493 version = task.last_version;
494 } else {
495 version = -1;
496 }
497 LOG.info("resubmitting task " + path);
498 task.incarnation++;
499 try {
500
501 if (ZKUtil.setData(this.watcher, path,
502 TaskState.TASK_UNASSIGNED.get(serverName),
503 version) == false) {
504 LOG.debug("failed to resubmit task " + path +
505 " version changed");
506 return false;
507 }
508 } catch (NoNodeException e) {
509 LOG.debug("failed to resubmit " + path + " task done");
510 return false;
511 } catch (KeeperException e) {
512 tot_mgr_resubmit_failed.incrementAndGet();
513 LOG.warn("failed to resubmit " + path, e);
514 return false;
515 }
516
517 if (directive != FORCE) {
518 task.unforcedResubmits++;
519 }
520 task.setUnassigned();
521 createRescanNode(Long.MAX_VALUE);
522 tot_mgr_resubmit.incrementAndGet();
523 return true;
524 }
525
526 private void resubmitOrFail(String path, ResubmitDirective directive) {
527 if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
528 setDone(path, FAILURE);
529 }
530 }
531
532 private void deleteNode(String path, Long retries) {
533 tot_mgr_node_delete_queued.incrementAndGet();
534 this.watcher.getRecoverableZooKeeper().getZooKeeper().
535 delete(path, -1, new DeleteAsyncCallback(),
536 retries);
537 }
538
539 private void deleteNodeSuccess(String path) {
540 Task task;
541 task = tasks.remove(path);
542 if (task == null) {
543 if (ZKSplitLog.isRescanNode(watcher, path)) {
544 tot_mgr_rescan_deleted.incrementAndGet();
545 }
546 tot_mgr_missing_state_in_delete.incrementAndGet();
547 LOG.debug("deleted task without in memory state " + path);
548 return;
549 }
550 tot_mgr_task_deleted.incrementAndGet();
551 }
552
553 private void deleteNodeFailure(String path) {
554 LOG.fatal("logic failure, failing to delete a node should never happen " +
555 "because delete has infinite retries");
556 return;
557 }
558
559
560
561
562
563
564 private void createRescanNode(long retries) {
565
566
567
568
569
570
571
572 this.watcher.getRecoverableZooKeeper().getZooKeeper().
573 create(ZKSplitLog.getRescanNode(watcher),
574 TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE,
575 CreateMode.EPHEMERAL_SEQUENTIAL,
576 new CreateRescanAsyncCallback(), new Long(retries));
577 }
578
579 private void createRescanSuccess(String path) {
580 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
581 tot_mgr_rescan.incrementAndGet();
582 getDataSetWatch(path, zkretries);
583 }
584
585 private void createRescanFailure() {
586 LOG.fatal("logic failure, rescan failure must not happen");
587 }
588
589
590
591
592
593
594 private Task createTaskIfAbsent(String path, TaskBatch batch) {
595 Task oldtask;
596 oldtask = tasks.putIfAbsent(path, new Task(batch));
597 if (oldtask != null && oldtask.isOrphan()) {
598 LOG.info("Previously orphan task " + path +
599 " is now being waited upon");
600 oldtask.setBatch(batch);
601 return (null);
602 }
603 return oldtask;
604 }
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624 void stopTrackingTasks(TaskBatch batch) {
625 for (Map.Entry<String, Task> e : tasks.entrySet()) {
626 String path = e.getKey();
627 Task t = e.getValue();
628 if (t.batch == batch) {
629 tasks.remove(path);
630 }
631 }
632 }
633
634 Task findOrCreateOrphanTask(String path) {
635 Task orphanTask = new Task(null);
636 Task task;
637 task = tasks.putIfAbsent(path, orphanTask);
638 if (task == null) {
639 LOG.info("creating orphan task " + path);
640 tot_mgr_orphan_task_acquired.incrementAndGet();
641 task = orphanTask;
642 }
643 return task;
644 }
645
646 @Override
647 public void nodeDataChanged(String path) {
648 if (tasks.get(path) != null || ZKSplitLog.isRescanNode(watcher, path)) {
649 getDataSetWatch(path, zkretries);
650 }
651 }
652
653 public void stop() {
654 if (timeoutMonitor != null) {
655 timeoutMonitor.interrupt();
656 }
657 }
658
659 private void lookForOrphans() {
660 List<String> orphans;
661 try {
662 orphans = ZKUtil.listChildrenNoWatch(this.watcher,
663 this.watcher.splitLogZNode);
664 if (orphans == null) {
665 LOG.warn("could not get children of " + this.watcher.splitLogZNode);
666 return;
667 }
668 } catch (KeeperException e) {
669 LOG.warn("could not get children of " + this.watcher.splitLogZNode +
670 " " + StringUtils.stringifyException(e));
671 return;
672 }
673 int rescan_nodes = 0;
674 for (String path : orphans) {
675 String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
676 if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
677 rescan_nodes++;
678 LOG.debug("found orphan rescan node " + path);
679 } else {
680 LOG.info("found orphan task " + path);
681 }
682 getDataSetWatch(nodepath, zkretries);
683 }
684 LOG.info("found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
685 rescan_nodes + " rescan nodes");
686 }
687
688
689
690
691
692
693
694
695 static class TaskBatch {
696 int installed;
697 int done;
698 int error;
699
700 @Override
701 public String toString() {
702 return ("installed = " + installed + " done = " + done + " error = "
703 + error);
704 }
705 }
706
707
708
709
710 static class Task {
711 long last_update;
712 int last_version;
713 String cur_worker_name;
714 TaskBatch batch;
715 boolean deleted;
716 int incarnation;
717 int unforcedResubmits;
718
719 @Override
720 public String toString() {
721 return ("last_update = " + last_update +
722 " last_version = " + last_version +
723 " cur_worker_name = " + cur_worker_name +
724 " deleted = " + deleted +
725 " incarnation = " + incarnation +
726 " resubmits = " + unforcedResubmits +
727 " batch = " + batch);
728 }
729
730 Task(TaskBatch tb) {
731 incarnation = 0;
732 last_version = -1;
733 deleted = false;
734 setBatch(tb);
735 setUnassigned();
736 }
737
738 public void setBatch(TaskBatch batch) {
739 if (batch != null && this.batch != null) {
740 LOG.fatal("logic error - batch being overwritten");
741 }
742 this.batch = batch;
743 if (batch != null) {
744 if (this.incarnation == 0) {
745 batch.installed++;
746 }
747 }
748 }
749
750 public boolean isOrphan() {
751 return (batch == null);
752 }
753
754 public boolean isUnassigned() {
755 return (last_update == -1);
756 }
757
758 public void heartbeat(long time, int version, String worker) {
759 last_version = version;
760 last_update = time;
761 cur_worker_name = worker;
762 }
763
764 public void setUnassigned() {
765 cur_worker_name = null;
766 last_update = -1;
767 }
768 }
769
770 void handleDeadWorker(String worker_name) {
771
772
773 synchronized (deadWorkersLock) {
774 if (deadWorkers == null) {
775 deadWorkers = new HashSet<String>(100);
776 }
777 deadWorkers.add(worker_name);
778 }
779 LOG.info("dead splitlog worker " + worker_name);
780 }
781
782
783
784
785
786 private class TimeoutMonitor extends Chore {
787 public TimeoutMonitor(final int period, Stoppable stopper) {
788 super("SplitLogManager Timeout Monitor", period, stopper);
789 }
790
791 @Override
792 protected void chore() {
793 int resubmitted = 0;
794 int unassigned = 0;
795 int tot = 0;
796 boolean found_assigned_task = false;
797 Set<String> localDeadWorkers;
798
799 synchronized (deadWorkersLock) {
800 localDeadWorkers = deadWorkers;
801 deadWorkers = null;
802 }
803
804 for (Map.Entry<String, Task> e : tasks.entrySet()) {
805 String path = e.getKey();
806 Task task = e.getValue();
807 String cur_worker = task.cur_worker_name;
808 tot++;
809
810
811
812
813
814 if (task.isUnassigned()) {
815 unassigned++;
816 continue;
817 }
818 found_assigned_task = true;
819 if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
820 tot_mgr_resubmit_dead_server_task.incrementAndGet();
821 if (resubmit(path, task, FORCE)) {
822 resubmitted++;
823 } else {
824 handleDeadWorker(cur_worker);
825 LOG.warn("Failed to resubmit task " + path + " owned by dead " +
826 cur_worker + ", will retry.");
827 }
828 } else if (resubmit(path, task, CHECK)) {
829 resubmitted++;
830 }
831 }
832 if (tot > 0) {
833 LOG.debug("total tasks = " + tot + " unassigned = " + unassigned);
834 }
835 if (resubmitted > 0) {
836 LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
837 }
838
839
840
841
842
843
844
845
846
847 if (tot > 0 && !found_assigned_task &&
848 ((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
849 unassignedTimeout)) {
850 createRescanNode(Long.MAX_VALUE);
851 tot_mgr_resubmit_unassigned.incrementAndGet();
852 LOG.debug("resubmitting unassigned task(s) after timeout");
853 }
854 }
855 }
856
857
858
859
860
861 class CreateAsyncCallback implements AsyncCallback.StringCallback {
862 private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
863
864 @Override
865 public void processResult(int rc, String path, Object ctx, String name) {
866 tot_mgr_node_create_result.incrementAndGet();
867 if (rc != 0) {
868 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
869 LOG.debug("found pre-existing znode " + path);
870 tot_mgr_node_already_exists.incrementAndGet();
871 } else {
872 Long retry_count = (Long)ctx;
873 LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
874 path + " retry=" + retry_count);
875 if (retry_count == 0) {
876 tot_mgr_node_create_err.incrementAndGet();
877 createNodeFailure(path);
878 } else {
879 tot_mgr_node_create_retry.incrementAndGet();
880 createNode(path, retry_count - 1);
881 }
882 return;
883 }
884 }
885 createNodeSuccess(path);
886 }
887 }
888
889
890
891
892
893 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
894 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
895
896 @Override
897 public void processResult(int rc, String path, Object ctx, byte[] data,
898 Stat stat) {
899 tot_mgr_get_data_result.incrementAndGet();
900 if (rc != 0) {
901 Long retry_count = (Long) ctx;
902 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
903 path + " retry=" + retry_count);
904 if (retry_count == 0) {
905 tot_mgr_get_data_err.incrementAndGet();
906 getDataSetWatchFailure(path);
907 } else {
908 tot_mgr_get_data_retry.incrementAndGet();
909 getDataSetWatch(path, retry_count - 1);
910 }
911 return;
912 }
913 getDataSetWatchSuccess(path, data, stat.getVersion());
914 return;
915 }
916 }
917
918
919
920
921
922 class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
923 private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
924
925 @Override
926 public void processResult(int rc, String path, Object ctx) {
927 tot_mgr_node_delete_result.incrementAndGet();
928 if (rc != 0) {
929 if (rc != KeeperException.Code.NONODE.intValue()) {
930 tot_mgr_node_delete_err.incrementAndGet();
931 Long retry_count = (Long) ctx;
932 LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
933 path + " retry=" + retry_count);
934 if (retry_count == 0) {
935 LOG.warn("delete failed " + path);
936 deleteNodeFailure(path);
937 } else {
938 deleteNode(path, retry_count - 1);
939 }
940 return;
941 } else {
942 LOG.debug(path
943 + " does not exist, either was never created or was deleted"
944 + " in earlier rounds, zkretries = " + (Long) ctx);
945 }
946 } else {
947 LOG.debug("deleted " + path);
948 }
949 deleteNodeSuccess(path);
950 }
951 }
952
953
954
955
956
957
958
959
960 class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
961 private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
962
963 @Override
964 public void processResult(int rc, String path, Object ctx, String name) {
965 if (rc != 0) {
966 Long retry_count = (Long)ctx;
967 LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
968 " retry=" + retry_count);
969 if (retry_count == 0) {
970 createRescanFailure();
971 } else {
972 createRescanNode(retry_count - 1);
973 }
974 return;
975 }
976
977 createRescanSuccess(name);
978 }
979 }
980
981
982
983
984
985
986
987
988
989 public boolean anyNewLogFiles(Path logdir, FileStatus[] logfiles)
990 throws IOException {
991 if (logdir == null) {
992 return false;
993 }
994 LOG.debug("re-listing " + logdir);
995 tot_mgr_relist_logdir.incrementAndGet();
996 FileStatus[] newfiles = FSUtils.listStatus(fs, logdir, null);
997 if (newfiles == null) {
998 return false;
999 }
1000 boolean matched;
1001 for (FileStatus newfile : newfiles) {
1002 matched = false;
1003 for (FileStatus origfile : logfiles) {
1004 if (origfile.equals(newfile)) {
1005 matched = true;
1006 break;
1007 }
1008 }
1009 if (matched == false) {
1010 LOG.warn("Discovered orphan hlog " + newfile + " after split." +
1011 " Maybe HRegionServer was not dead when we started");
1012 return true;
1013 }
1014 }
1015 return false;
1016 }
1017
1018
1019
1020
1021
1022
1023 static public interface TaskFinisher {
1024
1025
1026
1027 static public enum Status {
1028
1029
1030
1031 DONE(),
1032
1033
1034
1035 ERR();
1036 }
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046 public Status finish(String workerName, String taskname);
1047 }
1048 enum ResubmitDirective {
1049 CHECK(),
1050 FORCE();
1051 }
1052 enum TerminationStatus {
1053 SUCCESS(),
1054 FAILURE();
1055 }
1056 }