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 import java.util.concurrent.atomic.AtomicInteger;
37 import java.util.concurrent.locks.ReentrantLock;
38
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.classification.InterfaceAudience;
42 import org.apache.hadoop.conf.Configuration;
43 import org.apache.hadoop.fs.FileStatus;
44 import org.apache.hadoop.fs.FileSystem;
45 import org.apache.hadoop.fs.Path;
46 import org.apache.hadoop.fs.PathFilter;
47 import org.apache.hadoop.hbase.Chore;
48 import org.apache.hadoop.hbase.HRegionInfo;
49 import org.apache.hadoop.hbase.ServerName;
50 import org.apache.hadoop.hbase.SplitLogCounters;
51 import org.apache.hadoop.hbase.SplitLogTask;
52 import org.apache.hadoop.hbase.Stoppable;
53 import org.apache.hadoop.hbase.exceptions.DeserializationException;
54 import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
55 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
56 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
57 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.RegionStoreSequenceIds;
58 import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
59 import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
60 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
61 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
62 import org.apache.hadoop.hbase.util.FSUtils;
63 import org.apache.hadoop.hbase.util.Pair;
64 import org.apache.hadoop.hbase.util.Threads;
65 import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
66 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
67 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
68 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
69 import org.apache.hadoop.util.StringUtils;
70 import org.apache.zookeeper.AsyncCallback;
71 import org.apache.zookeeper.CreateMode;
72 import org.apache.zookeeper.KeeperException;
73 import org.apache.zookeeper.KeeperException.NoNodeException;
74 import org.apache.zookeeper.ZooDefs.Ids;
75 import org.apache.zookeeper.data.Stat;
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105 @InterfaceAudience.Private
106 public class SplitLogManager extends ZooKeeperListener {
107 private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
108
109 public static final int DEFAULT_TIMEOUT = 120000;
110 public static final int DEFAULT_ZK_RETRIES = 3;
111 public static final int DEFAULT_MAX_RESUBMIT = 3;
112 public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000);
113
114 private final Stoppable stopper;
115 private final MasterServices master;
116 private final ServerName serverName;
117 private final TaskFinisher taskFinisher;
118 private FileSystem fs;
119 private Configuration conf;
120
121 private long zkretries;
122 private long resubmit_threshold;
123 private long timeout;
124 private long unassignedTimeout;
125 private long lastTaskCreateTime = Long.MAX_VALUE;
126 public boolean ignoreZKDeleteForTesting = false;
127 private volatile long lastRecoveringNodeCreationTime = 0;
128
129
130 private long checkRecoveringTimeThreshold = 15000;
131 private final List<Pair<Set<ServerName>, Boolean>> failedRecoveringRegionDeletions = Collections
132 .synchronizedList(new ArrayList<Pair<Set<ServerName>, Boolean>>());
133
134
135
136
137
138 protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
139
140 final boolean distributedLogReplay;
141
142 private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
143 private TimeoutMonitor timeoutMonitor;
144
145 private volatile Set<ServerName> deadWorkers = null;
146 private final Object deadWorkersLock = new Object();
147
148 private Set<String> failedDeletions = null;
149
150
151
152
153
154
155
156
157
158
159
160
161
162 public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
163 Stoppable stopper, MasterServices master, ServerName serverName) {
164 this(zkw, conf, stopper, master, serverName, false, null);
165 }
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182 public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
183 Stoppable stopper, MasterServices master, ServerName serverName, boolean masterRecovery) {
184 this(zkw, conf, stopper, master, serverName, masterRecovery, new TaskFinisher() {
185 @Override
186 public Status finish(ServerName workerName, String logfile) {
187 try {
188 HLogSplitter.finishSplitLogFile(logfile, conf);
189 } catch (IOException e) {
190 LOG.warn("Could not finish splitting of log file " + logfile, e);
191 return Status.ERR;
192 }
193 return Status.DONE;
194 }
195 });
196 }
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211 public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
212 Stoppable stopper, MasterServices master,
213 ServerName serverName, boolean masterRecovery, TaskFinisher tf) {
214 super(zkw);
215 this.taskFinisher = tf;
216 this.conf = conf;
217 this.stopper = stopper;
218 this.master = master;
219 this.zkretries = conf.getLong("hbase.splitlog.zk.retries", DEFAULT_ZK_RETRIES);
220 this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit", DEFAULT_MAX_RESUBMIT);
221 this.timeout = conf.getInt("hbase.splitlog.manager.timeout", DEFAULT_TIMEOUT);
222 this.unassignedTimeout =
223 conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
224 this.distributedLogReplay = HLogSplitter.isDistributedLogReplay(conf);
225 LOG.info("Timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout +
226 ", distributedLogReplay=" + this.distributedLogReplay);
227
228 this.serverName = serverName;
229 this.timeoutMonitor = new TimeoutMonitor(
230 conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
231
232 this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
233
234 if (!masterRecovery) {
235 Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
236 + ".splitLogManagerTimeoutMonitor");
237 }
238
239 if (this.watcher != null) {
240 this.watcher.registerListener(this);
241 lookForOrphans();
242 }
243 }
244
245 private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
246 List<FileStatus> fileStatus = new ArrayList<FileStatus>();
247 for (Path hLogDir : logDirs) {
248 this.fs = hLogDir.getFileSystem(conf);
249 if (!fs.exists(hLogDir)) {
250 LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
251 continue;
252 }
253 FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, filter);
254 if (logfiles == null || logfiles.length == 0) {
255 LOG.info(hLogDir + " is empty dir, no logs to split");
256 } else {
257 for (FileStatus status : logfiles)
258 fileStatus.add(status);
259 }
260 }
261 FileStatus[] a = new FileStatus[fileStatus.size()];
262 return fileStatus.toArray(a);
263 }
264
265
266
267
268
269
270
271
272
273 public long splitLogDistributed(final Path logDir) throws IOException {
274 List<Path> logDirs = new ArrayList<Path>();
275 logDirs.add(logDir);
276 return splitLogDistributed(logDirs);
277 }
278
279
280
281
282
283
284
285
286
287
288
289 public long splitLogDistributed(final List<Path> logDirs) throws IOException {
290 if (logDirs.isEmpty()) {
291 return 0;
292 }
293 Set<ServerName> serverNames = new HashSet<ServerName>();
294 for (Path logDir : logDirs) {
295 try {
296 ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(logDir);
297 if (serverName != null) {
298 serverNames.add(serverName);
299 }
300 } catch (IllegalArgumentException e) {
301
302 LOG.warn("Cannot parse server name from " + logDir);
303 }
304 }
305 return splitLogDistributed(serverNames, logDirs, null);
306 }
307
308
309
310
311
312
313
314
315
316
317
318
319 public long splitLogDistributed(final Set<ServerName> serverNames, final List<Path> logDirs,
320 PathFilter filter) throws IOException {
321 MonitoredTask status = TaskMonitor.get().createStatus(
322 "Doing distributed log split in " + logDirs);
323 FileStatus[] logfiles = getFileList(logDirs, filter);
324 status.setStatus("Checking directory contents...");
325 LOG.debug("Scheduling batch of logs to split");
326 SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
327 LOG.info("started splitting " + logfiles.length + " logs in " + logDirs);
328 long t = EnvironmentEdgeManager.currentTimeMillis();
329 long totalSize = 0;
330 TaskBatch batch = new TaskBatch();
331 Boolean isMetaRecovery = (filter == null) ? null : false;
332 for (FileStatus lf : logfiles) {
333
334
335
336
337
338 totalSize += lf.getLen();
339 String pathToLog = FSUtils.removeRootPath(lf.getPath(), conf);
340 if (!enqueueSplitTask(pathToLog, batch)) {
341 throw new IOException("duplicate log split scheduled for " + lf.getPath());
342 }
343 }
344 waitForSplittingCompletion(batch, status);
345
346 if (filter == MasterFileSystem.META_FILTER
347
348
349 isMetaRecovery = true;
350 }
351 this.removeRecoveringRegionsFromZK(serverNames, isMetaRecovery);
352
353 if (batch.done != batch.installed) {
354 batch.isDead = true;
355 SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
356 LOG.warn("error while splitting logs in " + logDirs +
357 " installed = " + batch.installed + " but only " + batch.done + " done");
358 String msg = "error or interrupted while splitting logs in "
359 + logDirs + " Task = " + batch;
360 status.abort(msg);
361 throw new IOException(msg);
362 }
363 for(Path logDir: logDirs){
364 status.setStatus("Cleaning up log directory...");
365 try {
366 if (fs.exists(logDir) && !fs.delete(logDir, false)) {
367 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
368 }
369 } catch (IOException ioe) {
370 FileStatus[] files = fs.listStatus(logDir);
371 if (files != null && files.length > 0) {
372 LOG.warn("returning success without actually splitting and " +
373 "deleting all the log files in path " + logDir);
374 } else {
375 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
376 }
377 }
378 SplitLogCounters.tot_mgr_log_split_batch_success.incrementAndGet();
379 }
380 String msg = "finished splitting (more than or equal to) " + totalSize +
381 " bytes in " + batch.installed + " log files in " + logDirs + " in " +
382 (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms";
383 status.markComplete(msg);
384 LOG.info(msg);
385 return totalSize;
386 }
387
388
389
390
391
392
393
394
395 boolean enqueueSplitTask(String taskname, TaskBatch batch) {
396 SplitLogCounters.tot_mgr_log_split_start.incrementAndGet();
397
398
399 String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
400 lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
401 Task oldtask = createTaskIfAbsent(path, batch);
402 if (oldtask == null) {
403
404 createNode(path, zkretries);
405 return true;
406 }
407 return false;
408 }
409
410 private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {
411 synchronized (batch) {
412 while ((batch.done + batch.error) != batch.installed) {
413 try {
414 status.setStatus("Waiting for distributed tasks to finish. "
415 + " scheduled=" + batch.installed
416 + " done=" + batch.done
417 + " error=" + batch.error);
418 int remaining = batch.installed - (batch.done + batch.error);
419 int actual = activeTasks(batch);
420 if (remaining != actual) {
421 LOG.warn("Expected " + remaining
422 + " active tasks, but actually there are " + actual);
423 }
424 int remainingInZK = remainingTasksInZK();
425 if (remainingInZK >= 0 && actual > remainingInZK) {
426 LOG.warn("Expected at least" + actual
427 + " tasks in ZK, but actually there are " + remainingInZK);
428 }
429 if (remainingInZK == 0 || actual == 0) {
430 LOG.warn("No more task remaining (ZK or task map), splitting "
431 + "should have completed. Remaining tasks in ZK " + remainingInZK
432 + ", active tasks in map " + actual);
433 if (remainingInZK == 0 && actual == 0) {
434 return;
435 }
436 }
437 batch.wait(100);
438 if (stopper.isStopped()) {
439 LOG.warn("Stopped while waiting for log splits to be completed");
440 return;
441 }
442 } catch (InterruptedException e) {
443 LOG.warn("Interrupted while waiting for log splits to be completed");
444 Thread.currentThread().interrupt();
445 return;
446 }
447 }
448 }
449 }
450
451 private int activeTasks(final TaskBatch batch) {
452 int count = 0;
453 for (Task t: tasks.values()) {
454 if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
455 count++;
456 }
457 }
458 return count;
459 }
460
461 private int remainingTasksInZK() {
462 int count = 0;
463 try {
464 List<String> tasks =
465 ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
466 if (tasks != null) {
467 for (String t: tasks) {
468 if (!ZKSplitLog.isRescanNode(watcher, t)) {
469 count++;
470 }
471 }
472 }
473 } catch (KeeperException ke) {
474 LOG.warn("Failed to check remaining tasks", ke);
475 count = -1;
476 }
477 return count;
478 }
479
480
481
482
483
484
485
486
487 private void
488 removeRecoveringRegionsFromZK(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
489
490 if (!this.distributedLogReplay) {
491
492 return;
493 }
494
495 final String metaEncodeRegionName = HRegionInfo.FIRST_META_REGIONINFO.getEncodedName();
496 int count = 0;
497 Set<String> recoveredServerNameSet = new HashSet<String>();
498 if (serverNames != null) {
499 for (ServerName tmpServerName : serverNames) {
500 recoveredServerNameSet.add(tmpServerName.getServerName());
501 }
502 }
503
504 try {
505 this.recoveringRegionLock.lock();
506
507 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
508 if (tasks != null) {
509 for (String t : tasks) {
510 if (!ZKSplitLog.isRescanNode(watcher, t)) {
511 count++;
512 }
513 }
514 }
515 if (count == 0 && this.master.isInitialized()
516 && !this.master.getServerManager().areDeadServersInProgress()) {
517
518 deleteRecoveringRegionZNodes(null);
519
520
521 lastRecoveringNodeCreationTime = Long.MAX_VALUE;
522 } else if (!recoveredServerNameSet.isEmpty()) {
523
524 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
525 if (regions != null) {
526 for (String region : regions) {
527 if(isMetaRecovery != null) {
528 if ((isMetaRecovery && !region.equalsIgnoreCase(metaEncodeRegionName))
529 || (!isMetaRecovery && region.equalsIgnoreCase(metaEncodeRegionName))) {
530
531
532 continue;
533 }
534 }
535 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
536 List<String> failedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
537 if (failedServers == null || failedServers.isEmpty()) {
538 ZKUtil.deleteNode(watcher, nodePath);
539 continue;
540 }
541 if (recoveredServerNameSet.containsAll(failedServers)) {
542 ZKUtil.deleteNodeRecursively(watcher, nodePath);
543 } else {
544 for (String failedServer : failedServers) {
545 if (recoveredServerNameSet.contains(failedServer)) {
546 String tmpPath = ZKUtil.joinZNode(nodePath, failedServer);
547 ZKUtil.deleteNode(watcher, tmpPath);
548 }
549 }
550 }
551 }
552 }
553 }
554 } catch (KeeperException ke) {
555 LOG.warn("removeRecoveringRegionsFromZK got zookeeper exception. Will retry", ke);
556 if (serverNames != null && !serverNames.isEmpty()) {
557 this.failedRecoveringRegionDeletions.add(new Pair<Set<ServerName>, Boolean>(serverNames,
558 isMetaRecovery));
559 }
560 } finally {
561 this.recoveringRegionLock.unlock();
562 }
563 }
564
565
566
567
568
569
570
571 void removeStaleRecoveringRegionsFromZK(final Set<ServerName> failedServers)
572 throws KeeperException {
573
574 if (!this.distributedLogReplay) {
575
576
577 ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode);
578
579 return;
580 }
581
582 Set<String> knownFailedServers = new HashSet<String>();
583 if (failedServers != null) {
584 for (ServerName tmpServerName : failedServers) {
585 knownFailedServers.add(tmpServerName.getServerName());
586 }
587 }
588
589 this.recoveringRegionLock.lock();
590 try {
591 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
592 if (tasks != null) {
593 for (String t : tasks) {
594 byte[] data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
595 if (data != null) {
596 SplitLogTask slt = null;
597 try {
598 slt = SplitLogTask.parseFrom(data);
599 } catch (DeserializationException e) {
600 LOG.warn("Failed parse data for znode " + t, e);
601 }
602 if (slt != null && slt.isDone()) {
603 continue;
604 }
605 }
606
607 t = ZKSplitLog.getFileName(t);
608 ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(new Path(t));
609 if (serverName != null) {
610 knownFailedServers.add(serverName.getServerName());
611 } else {
612 LOG.warn("Found invalid WAL log file name:" + t);
613 }
614 }
615 }
616
617
618 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
619 if (regions != null) {
620 for (String region : regions) {
621 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
622 List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
623 if (regionFailedServers == null || regionFailedServers.isEmpty()) {
624 ZKUtil.deleteNode(watcher, nodePath);
625 continue;
626 }
627 boolean needMoreRecovery = false;
628 for (String tmpFailedServer : regionFailedServers) {
629 if (knownFailedServers.contains(tmpFailedServer)) {
630 needMoreRecovery = true;
631 break;
632 }
633 }
634 if (!needMoreRecovery) {
635 ZKUtil.deleteNodeRecursively(watcher, nodePath);
636 }
637 }
638 }
639 } finally {
640 this.recoveringRegionLock.unlock();
641 }
642 }
643
644 private void deleteRecoveringRegionZNodes(List<String> regions) {
645 try {
646 if (regions == null) {
647
648 LOG.info("Garbage collecting all recovering regions.");
649 ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode);
650 } else {
651 for (String curRegion : regions) {
652 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, curRegion);
653 ZKUtil.deleteNodeRecursively(watcher, nodePath);
654 }
655 }
656 } catch (KeeperException e) {
657 LOG.warn("Cannot remove recovering regions from ZooKeeper", e);
658 }
659 }
660
661 private void setDone(String path, TerminationStatus status) {
662 Task task = tasks.get(path);
663 if (task == null) {
664 if (!ZKSplitLog.isRescanNode(watcher, path)) {
665 SplitLogCounters.tot_mgr_unacquired_orphan_done.incrementAndGet();
666 LOG.debug("unacquired orphan task is done " + path);
667 }
668 } else {
669 synchronized (task) {
670 if (task.status == IN_PROGRESS) {
671 if (status == SUCCESS) {
672 SplitLogCounters.tot_mgr_log_split_success.incrementAndGet();
673 LOG.info("Done splitting " + path);
674 } else {
675 SplitLogCounters.tot_mgr_log_split_err.incrementAndGet();
676 LOG.warn("Error splitting " + path);
677 }
678 task.status = status;
679 if (task.batch != null) {
680 synchronized (task.batch) {
681 if (status == SUCCESS) {
682 task.batch.done++;
683 } else {
684 task.batch.error++;
685 }
686 task.batch.notify();
687 }
688 }
689 }
690 }
691 }
692
693
694
695
696
697 deleteNode(path, zkretries);
698 return;
699 }
700
701 private void createNode(String path, Long retry_count) {
702 SplitLogTask slt = new SplitLogTask.Unassigned(serverName);
703 ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(), retry_count);
704 SplitLogCounters.tot_mgr_node_create_queued.incrementAndGet();
705 return;
706 }
707
708 private void createNodeSuccess(String path) {
709 LOG.debug("put up splitlog task at znode " + path);
710 getDataSetWatch(path, zkretries);
711 }
712
713 private void createNodeFailure(String path) {
714
715 LOG.warn("failed to create task node" + path);
716 setDone(path, FAILURE);
717 }
718
719
720 private void getDataSetWatch(String path, Long retry_count) {
721 this.watcher.getRecoverableZooKeeper().getZooKeeper().
722 getData(path, this.watcher,
723 new GetDataAsyncCallback(true), retry_count);
724 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
725 }
726
727 private void tryGetDataSetWatch(String path) {
728
729 this.watcher.getRecoverableZooKeeper().getZooKeeper().
730 getData(path, this.watcher,
731 new GetDataAsyncCallback(false), Long.valueOf(-1)
732 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
733 }
734
735 private void getDataSetWatchSuccess(String path, byte[] data, int version)
736 throws DeserializationException {
737 if (data == null) {
738 if (version == Integer.MIN_VALUE) {
739
740 setDone(path, SUCCESS);
741 return;
742 }
743 SplitLogCounters.tot_mgr_null_data.incrementAndGet();
744 LOG.fatal("logic error - got null data " + path);
745 setDone(path, FAILURE);
746 return;
747 }
748 data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
749 SplitLogTask slt = SplitLogTask.parseFrom(data);
750 if (slt.isUnassigned()) {
751 LOG.debug("task not yet acquired " + path + " ver = " + version);
752 handleUnassignedTask(path);
753 } else if (slt.isOwned()) {
754 heartbeat(path, version, slt.getServerName());
755 } else if (slt.isResigned()) {
756 LOG.info("task " + path + " entered state: " + slt.toString());
757 resubmitOrFail(path, FORCE);
758 } else if (slt.isDone()) {
759 LOG.info("task " + path + " entered state: " + slt.toString());
760 if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
761 if (taskFinisher.finish(slt.getServerName(), ZKSplitLog.getFileName(path)) == Status.DONE) {
762 setDone(path, SUCCESS);
763 } else {
764 resubmitOrFail(path, CHECK);
765 }
766 } else {
767 setDone(path, SUCCESS);
768 }
769 } else if (slt.isErr()) {
770 LOG.info("task " + path + " entered state: " + slt.toString());
771 resubmitOrFail(path, CHECK);
772 } else {
773 LOG.fatal("logic error - unexpected zk state for path = " + path + " data = " + slt.toString());
774 setDone(path, FAILURE);
775 }
776 }
777
778 private void getDataSetWatchFailure(String path) {
779 LOG.warn("failed to set data watch " + path);
780 setDone(path, FAILURE);
781 }
782
783
784
785
786
787
788
789
790
791
792 private void handleUnassignedTask(String path) {
793 if (ZKSplitLog.isRescanNode(watcher, path)) {
794 return;
795 }
796 Task task = findOrCreateOrphanTask(path);
797 if (task.isOrphan() && (task.incarnation == 0)) {
798 LOG.info("resubmitting unassigned orphan task " + path);
799
800
801 resubmit(path, task, FORCE);
802 }
803 }
804
805
806
807
808
809
810
811 private boolean needAbandonRetries(int statusCode, String action) {
812 if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
813 LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
814 + "action=" + action);
815 return true;
816 }
817 return false;
818 }
819
820 private void heartbeat(String path, int new_version, ServerName workerName) {
821 Task task = findOrCreateOrphanTask(path);
822 if (new_version != task.last_version) {
823 if (task.isUnassigned()) {
824 LOG.info("task " + path + " acquired by " + workerName);
825 }
826 task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(), new_version, workerName);
827 SplitLogCounters.tot_mgr_heartbeat.incrementAndGet();
828 } else {
829
830
831
832
833 }
834 return;
835 }
836
837 private boolean resubmit(String path, Task task, ResubmitDirective directive) {
838
839 if (task.status != IN_PROGRESS) {
840 return false;
841 }
842 int version;
843 if (directive != FORCE) {
844
845
846
847
848
849 final long time = EnvironmentEdgeManager.currentTimeMillis() - task.last_update;
850 final boolean alive = master.getServerManager() != null ?
851 master.getServerManager().isServerOnline(task.cur_worker_name) : true;
852 if (alive && time < timeout) {
853 LOG.trace("Skipping the resubmit of " + task.toString() + " because the server " +
854 task.cur_worker_name + " is not marked as dead, we waited for " + time +
855 " while the timeout is " + timeout);
856 return false;
857 }
858 if (task.unforcedResubmits.get() >= resubmit_threshold) {
859 if (!task.resubmitThresholdReached) {
860 task.resubmitThresholdReached = true;
861 SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
862 LOG.info("Skipping resubmissions of task " + path +
863 " because threshold " + resubmit_threshold + " reached");
864 }
865 return false;
866 }
867
868 version = task.last_version;
869 } else {
870 SplitLogCounters.tot_mgr_resubmit_force.incrementAndGet();
871 version = -1;
872 }
873 LOG.info("resubmitting task " + path);
874 task.incarnation++;
875 try {
876
877 SplitLogTask slt = new SplitLogTask.Unassigned(this.serverName);
878 if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
879 LOG.debug("failed to resubmit task " + path +
880 " version changed");
881 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
882 return false;
883 }
884 } catch (NoNodeException e) {
885 LOG.warn("failed to resubmit because znode doesn't exist " + path +
886 " task done (or forced done by removing the znode)");
887 try {
888 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
889 } catch (DeserializationException e1) {
890 LOG.debug("Failed to re-resubmit task " + path + " because of deserialization issue", e1);
891 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
892 return false;
893 }
894 return false;
895 } catch (KeeperException.BadVersionException e) {
896 LOG.debug("failed to resubmit task " + path + " version changed");
897 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
898 return false;
899 } catch (KeeperException e) {
900 SplitLogCounters.tot_mgr_resubmit_failed.incrementAndGet();
901 LOG.warn("failed to resubmit " + path, e);
902 return false;
903 }
904
905 if (directive != FORCE) {
906 task.unforcedResubmits.incrementAndGet();
907 }
908 task.setUnassigned();
909 createRescanNode(Long.MAX_VALUE);
910 SplitLogCounters.tot_mgr_resubmit.incrementAndGet();
911 return true;
912 }
913
914 private void resubmitOrFail(String path, ResubmitDirective directive) {
915 if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
916 setDone(path, FAILURE);
917 }
918 }
919
920 private void deleteNode(String path, Long retries) {
921 SplitLogCounters.tot_mgr_node_delete_queued.incrementAndGet();
922
923
924
925 this.watcher.getRecoverableZooKeeper().getZooKeeper().
926 delete(path, -1, new DeleteAsyncCallback(),
927 retries);
928 }
929
930 private void deleteNodeSuccess(String path) {
931 if (ignoreZKDeleteForTesting) {
932 return;
933 }
934 Task task;
935 task = tasks.remove(path);
936 if (task == null) {
937 if (ZKSplitLog.isRescanNode(watcher, path)) {
938 SplitLogCounters.tot_mgr_rescan_deleted.incrementAndGet();
939 }
940 SplitLogCounters.tot_mgr_missing_state_in_delete.incrementAndGet();
941 LOG.debug("deleted task without in memory state " + path);
942 return;
943 }
944 synchronized (task) {
945 task.status = DELETED;
946 task.notify();
947 }
948 SplitLogCounters.tot_mgr_task_deleted.incrementAndGet();
949 }
950
951 private void deleteNodeFailure(String path) {
952 LOG.info("Failed to delete node " + path + " and will retry soon.");
953 return;
954 }
955
956
957
958
959
960
961 private void createRescanNode(long retries) {
962
963
964
965
966
967
968
969 lastTaskCreateTime = EnvironmentEdgeManager.currentTimeMillis();
970 SplitLogTask slt = new SplitLogTask.Done(this.serverName);
971 this.watcher.getRecoverableZooKeeper().getZooKeeper().
972 create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(),
973 Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL,
974 new CreateRescanAsyncCallback(), Long.valueOf(retries));
975 }
976
977 private void createRescanSuccess(String path) {
978 SplitLogCounters.tot_mgr_rescan.incrementAndGet();
979 getDataSetWatch(path, zkretries);
980 }
981
982 private void createRescanFailure() {
983 LOG.fatal("logic failure, rescan failure must not happen");
984 }
985
986
987
988
989
990
991 private Task createTaskIfAbsent(String path, TaskBatch batch) {
992 Task oldtask;
993
994
995 Task newtask = new Task();
996 newtask.batch = batch;
997 oldtask = tasks.putIfAbsent(path, newtask);
998 if (oldtask == null) {
999 batch.installed++;
1000 return null;
1001 }
1002
1003 synchronized (oldtask) {
1004 if (oldtask.isOrphan()) {
1005 if (oldtask.status == SUCCESS) {
1006
1007
1008
1009
1010 return (null);
1011 }
1012 if (oldtask.status == IN_PROGRESS) {
1013 oldtask.batch = batch;
1014 batch.installed++;
1015 LOG.debug("Previously orphan task " + path + " is now being waited upon");
1016 return null;
1017 }
1018 while (oldtask.status == FAILURE) {
1019 LOG.debug("wait for status of task " + path + " to change to DELETED");
1020 SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
1021 try {
1022 oldtask.wait();
1023 } catch (InterruptedException e) {
1024 Thread.currentThread().interrupt();
1025 LOG.warn("Interrupted when waiting for znode delete callback");
1026
1027 break;
1028 }
1029 }
1030 if (oldtask.status != DELETED) {
1031 LOG.warn("Failure because previously failed task" +
1032 " state still present. Waiting for znode delete callback" +
1033 " path=" + path);
1034 return oldtask;
1035 }
1036
1037 Task t = tasks.putIfAbsent(path, newtask);
1038 if (t == null) {
1039 batch.installed++;
1040 return null;
1041 }
1042 LOG.fatal("Logic error. Deleted task still present in tasks map");
1043 assert false : "Deleted task still present in tasks map";
1044 return t;
1045 }
1046 LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
1047 return oldtask;
1048 }
1049 }
1050
1051 Task findOrCreateOrphanTask(String path) {
1052 Task orphanTask = new Task();
1053 Task task;
1054 task = tasks.putIfAbsent(path, orphanTask);
1055 if (task == null) {
1056 LOG.info("creating orphan task " + path);
1057 SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
1058 task = orphanTask;
1059 }
1060 return task;
1061 }
1062
1063 @Override
1064 public void nodeDataChanged(String path) {
1065 Task task;
1066 task = tasks.get(path);
1067 if (task != null || ZKSplitLog.isRescanNode(watcher, path)) {
1068 if (task != null) {
1069 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
1070 }
1071 getDataSetWatch(path, zkretries);
1072 }
1073 }
1074
1075 public void stop() {
1076 if (timeoutMonitor != null) {
1077 timeoutMonitor.interrupt();
1078 }
1079 }
1080
1081 private void lookForOrphans() {
1082 List<String> orphans;
1083 try {
1084 orphans = ZKUtil.listChildrenNoWatch(this.watcher,
1085 this.watcher.splitLogZNode);
1086 if (orphans == null) {
1087 LOG.warn("could not get children of " + this.watcher.splitLogZNode);
1088 return;
1089 }
1090 } catch (KeeperException e) {
1091 LOG.warn("could not get children of " + this.watcher.splitLogZNode +
1092 " " + StringUtils.stringifyException(e));
1093 return;
1094 }
1095 int rescan_nodes = 0;
1096 for (String path : orphans) {
1097 String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
1098 if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
1099 rescan_nodes++;
1100 LOG.debug("found orphan rescan node " + path);
1101 } else {
1102 LOG.info("found orphan task " + path);
1103 }
1104 getDataSetWatch(nodepath, zkretries);
1105 }
1106 LOG.info("Found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
1107 rescan_nodes + " rescan nodes");
1108 }
1109
1110
1111
1112
1113
1114
1115
1116 void markRegionsRecoveringInZK(final ServerName serverName, Set<HRegionInfo> userRegions)
1117 throws KeeperException {
1118 if (userRegions == null || !this.distributedLogReplay) {
1119 return;
1120 }
1121
1122 try {
1123 this.recoveringRegionLock.lock();
1124
1125 this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTimeMillis();
1126
1127 for (HRegionInfo region : userRegions) {
1128 String regionEncodeName = region.getEncodedName();
1129 long retries = this.zkretries;
1130
1131 do {
1132 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName);
1133 long lastRecordedFlushedSequenceId = -1;
1134 try {
1135 long lastSequenceId = this.master.getServerManager().getLastFlushedSequenceId(
1136 regionEncodeName.getBytes());
1137
1138
1139
1140
1141
1142 byte[] data = ZKUtil.getData(this.watcher, nodePath);
1143 if (data == null) {
1144 ZKUtil.createSetData(this.watcher, nodePath,
1145 ZKUtil.positionToByteArray(lastSequenceId));
1146 } else {
1147 lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
1148 if (lastRecordedFlushedSequenceId < lastSequenceId) {
1149
1150 ZKUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
1151 }
1152 }
1153
1154 nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName());
1155 if (lastSequenceId <= lastRecordedFlushedSequenceId) {
1156
1157 lastSequenceId = lastRecordedFlushedSequenceId;
1158 }
1159 ZKUtil.createSetData(this.watcher, nodePath,
1160 ZKUtil.regionSequenceIdsToByteArray(lastSequenceId, null));
1161 LOG.debug("Mark region " + regionEncodeName + " recovering from failed region server "
1162 + serverName);
1163
1164
1165 break;
1166 } catch (KeeperException e) {
1167
1168 if (retries <= 1) {
1169 throw e;
1170 }
1171
1172 try {
1173 Thread.sleep(20);
1174 } catch (Exception ignoreE) {
1175
1176 }
1177 }
1178 } while ((--retries) > 0 && (!this.stopper.isStopped()));
1179 }
1180 } finally {
1181 this.recoveringRegionLock.unlock();
1182 }
1183 }
1184
1185
1186
1187
1188
1189 public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) {
1190 long lastRecordedFlushedSequenceId = -1l;
1191 try {
1192 lastRecordedFlushedSequenceId = ZKUtil.parseHLogPositionFrom(bytes);
1193 } catch (DeserializationException e) {
1194 lastRecordedFlushedSequenceId = -1l;
1195 LOG.warn("Can't parse last flushed sequence Id", e);
1196 }
1197 return lastRecordedFlushedSequenceId;
1198 }
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208 public static boolean
1209 isRegionMarkedRecoveringInZK(ZooKeeperWatcher zkw, String regionEncodedName)
1210 throws KeeperException {
1211 boolean result = false;
1212 String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, regionEncodedName);
1213
1214 byte[] node = ZKUtil.getDataAndWatch(zkw, nodePath);
1215 if (node != null) {
1216 result = true;
1217 }
1218 return result;
1219 }
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229 public static RegionStoreSequenceIds getRegionFlushedSequenceId(ZooKeeperWatcher zkw,
1230 String serverName, String encodedRegionName) throws IOException {
1231
1232
1233
1234
1235
1236
1237
1238 RegionStoreSequenceIds result = null;
1239 String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
1240 nodePath = ZKUtil.joinZNode(nodePath, serverName);
1241 try {
1242 byte[] data = ZKUtil.getData(zkw, nodePath);
1243 if (data != null) {
1244 result = ZKUtil.parseRegionStoreSequenceIds(data);
1245 }
1246 } catch (KeeperException e) {
1247 throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server="
1248 + serverName + "; region=" + encodedRegionName, e);
1249 } catch (DeserializationException e) {
1250 LOG.warn("Can't parse last flushed sequence Id from znode:" + nodePath, e);
1251 }
1252 return result;
1253 }
1254
1255
1256
1257
1258
1259
1260
1261 static class TaskBatch {
1262 int installed = 0;
1263 int done = 0;
1264 int error = 0;
1265 volatile boolean isDead = false;
1266
1267 @Override
1268 public String toString() {
1269 return ("installed = " + installed + " done = " + done + " error = " + error);
1270 }
1271 }
1272
1273
1274
1275
1276 static class Task {
1277 volatile long last_update;
1278 volatile int last_version;
1279 volatile ServerName cur_worker_name;
1280 volatile TaskBatch batch;
1281 volatile TerminationStatus status;
1282 volatile int incarnation;
1283 final AtomicInteger unforcedResubmits = new AtomicInteger();
1284 volatile boolean resubmitThresholdReached;
1285
1286 @Override
1287 public String toString() {
1288 return ("last_update = " + last_update +
1289 " last_version = " + last_version +
1290 " cur_worker_name = " + cur_worker_name +
1291 " status = " + status +
1292 " incarnation = " + incarnation +
1293 " resubmits = " + unforcedResubmits.get() +
1294 " batch = " + batch);
1295 }
1296
1297 Task() {
1298 incarnation = 0;
1299 last_version = -1;
1300 status = IN_PROGRESS;
1301 setUnassigned();
1302 }
1303
1304 public boolean isOrphan() {
1305 return (batch == null || batch.isDead);
1306 }
1307
1308 public boolean isUnassigned() {
1309 return (cur_worker_name == null);
1310 }
1311
1312 public void heartbeatNoDetails(long time) {
1313 last_update = time;
1314 }
1315
1316 public void heartbeat(long time, int version, ServerName worker) {
1317 last_version = version;
1318 last_update = time;
1319 cur_worker_name = worker;
1320 }
1321
1322 public void setUnassigned() {
1323 cur_worker_name = null;
1324 last_update = -1;
1325 }
1326 }
1327
1328 void handleDeadWorker(ServerName workerName) {
1329
1330
1331 synchronized (deadWorkersLock) {
1332 if (deadWorkers == null) {
1333 deadWorkers = new HashSet<ServerName>(100);
1334 }
1335 deadWorkers.add(workerName);
1336 }
1337 LOG.info("dead splitlog worker " + workerName);
1338 }
1339
1340 void handleDeadWorkers(Set<ServerName> serverNames) {
1341 synchronized (deadWorkersLock) {
1342 if (deadWorkers == null) {
1343 deadWorkers = new HashSet<ServerName>(100);
1344 }
1345 deadWorkers.addAll(serverNames);
1346 }
1347 LOG.info("dead splitlog workers " + serverNames);
1348 }
1349
1350
1351
1352
1353
1354 private class TimeoutMonitor extends Chore {
1355 private long lastLog = 0;
1356
1357 public TimeoutMonitor(final int period, Stoppable stopper) {
1358 super("SplitLogManager Timeout Monitor", period, stopper);
1359 }
1360
1361 @Override
1362 protected void chore() {
1363 int resubmitted = 0;
1364 int unassigned = 0;
1365 int tot = 0;
1366 boolean found_assigned_task = false;
1367 Set<ServerName> localDeadWorkers;
1368
1369 synchronized (deadWorkersLock) {
1370 localDeadWorkers = deadWorkers;
1371 deadWorkers = null;
1372 }
1373
1374 for (Map.Entry<String, Task> e : tasks.entrySet()) {
1375 String path = e.getKey();
1376 Task task = e.getValue();
1377 ServerName cur_worker = task.cur_worker_name;
1378 tot++;
1379
1380
1381
1382
1383
1384 if (task.isUnassigned()) {
1385 unassigned++;
1386 continue;
1387 }
1388 found_assigned_task = true;
1389 if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
1390 SplitLogCounters.tot_mgr_resubmit_dead_server_task.incrementAndGet();
1391 if (resubmit(path, task, FORCE)) {
1392 resubmitted++;
1393 } else {
1394 handleDeadWorker(cur_worker);
1395 LOG.warn("Failed to resubmit task " + path + " owned by dead " +
1396 cur_worker + ", will retry.");
1397 }
1398 } else if (resubmit(path, task, CHECK)) {
1399 resubmitted++;
1400 }
1401 }
1402 if (tot > 0) {
1403 long now = EnvironmentEdgeManager.currentTimeMillis();
1404 if (now > lastLog + 5000) {
1405 lastLog = now;
1406 LOG.info("total tasks = " + tot + " unassigned = " + unassigned + " tasks=" + tasks);
1407 }
1408 }
1409 if (resubmitted > 0) {
1410 LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
1411 }
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421 if (tot > 0 && !found_assigned_task &&
1422 ((EnvironmentEdgeManager.currentTimeMillis() - lastTaskCreateTime) >
1423 unassignedTimeout)) {
1424 for (Map.Entry<String, Task> e : tasks.entrySet()) {
1425 String path = e.getKey();
1426 Task task = e.getValue();
1427
1428
1429
1430
1431 if (task.isUnassigned() && (task.status != FAILURE)) {
1432
1433 tryGetDataSetWatch(path);
1434 }
1435 }
1436 createRescanNode(Long.MAX_VALUE);
1437 SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
1438 LOG.debug("resubmitting unassigned task(s) after timeout");
1439 }
1440
1441
1442 if (failedDeletions.size() > 0) {
1443 List<String> tmpPaths = new ArrayList<String>(failedDeletions);
1444 for (String tmpPath : tmpPaths) {
1445
1446 deleteNode(tmpPath, zkretries);
1447 }
1448 failedDeletions.removeAll(tmpPaths);
1449 }
1450
1451
1452 long timeInterval = EnvironmentEdgeManager.currentTimeMillis()
1453 - lastRecoveringNodeCreationTime;
1454 if (!failedRecoveringRegionDeletions.isEmpty()
1455 || (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) {
1456
1457 if (!failedRecoveringRegionDeletions.isEmpty()) {
1458 List<Pair<Set<ServerName>, Boolean>> previouslyFailedDeletions =
1459 new ArrayList<Pair<Set<ServerName>, Boolean>>(failedRecoveringRegionDeletions);
1460 failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletions);
1461 for (Pair<Set<ServerName>, Boolean> failedDeletion : previouslyFailedDeletions) {
1462 removeRecoveringRegionsFromZK(failedDeletion.getFirst(), failedDeletion.getSecond());
1463 }
1464 } else {
1465 removeRecoveringRegionsFromZK(null, null);
1466 }
1467 }
1468 }
1469 }
1470
1471
1472
1473
1474
1475 class CreateAsyncCallback implements AsyncCallback.StringCallback {
1476 private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
1477
1478 @Override
1479 public void processResult(int rc, String path, Object ctx, String name) {
1480 SplitLogCounters.tot_mgr_node_create_result.incrementAndGet();
1481 if (rc != 0) {
1482 if (needAbandonRetries(rc, "Create znode " + path)) {
1483 createNodeFailure(path);
1484 return;
1485 }
1486 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
1487
1488
1489
1490
1491
1492
1493 LOG.debug("found pre-existing znode " + path);
1494 SplitLogCounters.tot_mgr_node_already_exists.incrementAndGet();
1495 } else {
1496 Long retry_count = (Long)ctx;
1497 LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
1498 path + " remaining retries=" + retry_count);
1499 if (retry_count == 0) {
1500 SplitLogCounters.tot_mgr_node_create_err.incrementAndGet();
1501 createNodeFailure(path);
1502 } else {
1503 SplitLogCounters.tot_mgr_node_create_retry.incrementAndGet();
1504 createNode(path, retry_count - 1);
1505 }
1506 return;
1507 }
1508 }
1509 createNodeSuccess(path);
1510 }
1511 }
1512
1513
1514
1515
1516
1517 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
1518 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
1519 private boolean completeTaskOnNoNode;
1520
1521
1522
1523
1524
1525
1526
1527 public GetDataAsyncCallback(boolean completeTaskOnNoNode) {
1528 this.completeTaskOnNoNode = completeTaskOnNoNode;
1529 }
1530
1531 @Override
1532 public void processResult(int rc, String path, Object ctx, byte[] data,
1533 Stat stat) {
1534 SplitLogCounters.tot_mgr_get_data_result.incrementAndGet();
1535 if (rc != 0) {
1536 if (needAbandonRetries(rc, "GetData from znode " + path)) {
1537 return;
1538 }
1539 if (rc == KeeperException.Code.NONODE.intValue()) {
1540 SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
1541 LOG.warn("task znode " + path + " vanished.");
1542 if (completeTaskOnNoNode) {
1543
1544
1545
1546 try {
1547 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
1548 } catch (DeserializationException e) {
1549 LOG.warn("Deserialization problem", e);
1550 }
1551 }
1552 return;
1553 }
1554 Long retry_count = (Long) ctx;
1555
1556 if (retry_count < 0) {
1557 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1558 path + ". Ignoring error. No error handling. No retrying.");
1559 return;
1560 }
1561 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1562 path + " remaining retries=" + retry_count);
1563 if (retry_count == 0) {
1564 SplitLogCounters.tot_mgr_get_data_err.incrementAndGet();
1565 getDataSetWatchFailure(path);
1566 } else {
1567 SplitLogCounters.tot_mgr_get_data_retry.incrementAndGet();
1568 getDataSetWatch(path, retry_count - 1);
1569 }
1570 return;
1571 }
1572 try {
1573 getDataSetWatchSuccess(path, data, stat.getVersion());
1574 } catch (DeserializationException e) {
1575 LOG.warn("Deserialization problem", e);
1576 }
1577 return;
1578 }
1579 }
1580
1581
1582
1583
1584
1585 class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
1586 private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
1587
1588 @Override
1589 public void processResult(int rc, String path, Object ctx) {
1590 SplitLogCounters.tot_mgr_node_delete_result.incrementAndGet();
1591 if (rc != 0) {
1592 if (needAbandonRetries(rc, "Delete znode " + path)) {
1593 failedDeletions.add(path);
1594 return;
1595 }
1596 if (rc != KeeperException.Code.NONODE.intValue()) {
1597 SplitLogCounters.tot_mgr_node_delete_err.incrementAndGet();
1598 Long retry_count = (Long) ctx;
1599 LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
1600 path + " remaining retries=" + retry_count);
1601 if (retry_count == 0) {
1602 LOG.warn("delete failed " + path);
1603 failedDeletions.add(path);
1604 deleteNodeFailure(path);
1605 } else {
1606 deleteNode(path, retry_count - 1);
1607 }
1608 return;
1609 } else {
1610 LOG.info(path +
1611 " does not exist. Either was created but deleted behind our" +
1612 " back by another pending delete OR was deleted" +
1613 " in earlier retry rounds. zkretries = " + (Long) ctx);
1614 }
1615 } else {
1616 LOG.debug("deleted " + path);
1617 }
1618 deleteNodeSuccess(path);
1619 }
1620 }
1621
1622
1623
1624
1625
1626
1627
1628
1629 class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
1630 private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
1631
1632 @Override
1633 public void processResult(int rc, String path, Object ctx, String name) {
1634 if (rc != 0) {
1635 if (needAbandonRetries(rc, "CreateRescan znode " + path)) {
1636 return;
1637 }
1638 Long retry_count = (Long)ctx;
1639 LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
1640 " remaining retries=" + retry_count);
1641 if (retry_count == 0) {
1642 createRescanFailure();
1643 } else {
1644 createRescanNode(retry_count - 1);
1645 }
1646 return;
1647 }
1648
1649 createRescanSuccess(name);
1650 }
1651 }
1652
1653
1654
1655
1656
1657
1658
1659 public interface TaskFinisher {
1660
1661
1662
1663 enum Status {
1664
1665
1666
1667 DONE(),
1668
1669
1670
1671 ERR();
1672 }
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682 Status finish(ServerName workerName, String taskname);
1683 }
1684
1685 enum ResubmitDirective {
1686 CHECK(),
1687 FORCE();
1688 }
1689
1690 enum TerminationStatus {
1691 IN_PROGRESS("in_progress"),
1692 SUCCESS("success"),
1693 FAILURE("failure"),
1694 DELETED("deleted");
1695
1696 String statusMsg;
1697 TerminationStatus(String msg) {
1698 statusMsg = msg;
1699 }
1700
1701 @Override
1702 public String toString() {
1703 return statusMsg;
1704 }
1705 }
1706 }