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