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.locks.ReentrantLock;
37
38 import org.apache.commons.logging.Log;
39 import org.apache.commons.logging.LogFactory;
40 import org.apache.hadoop.classification.InterfaceAudience;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.fs.FileStatus;
43 import org.apache.hadoop.fs.FileSystem;
44 import org.apache.hadoop.fs.Path;
45 import org.apache.hadoop.fs.PathFilter;
46 import org.apache.hadoop.hbase.Chore;
47 import org.apache.hadoop.hbase.HConstants;
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 lastNodeCreateTime = 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 LOG.info("timeout=" + timeout + ", unassigned timeout=" + unassignedTimeout);
225
226 this.serverName = serverName;
227 this.timeoutMonitor = new TimeoutMonitor(
228 conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000), stopper);
229
230 this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
231 this.distributedLogReplay = this.conf.getBoolean(HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
232 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
233 LOG.info("distributedLogReplay = " + this.distributedLogReplay);
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 return;
577 }
578
579 Set<String> knownFailedServers = new HashSet<String>();
580 if (failedServers != null) {
581 for (ServerName tmpServerName : failedServers) {
582 knownFailedServers.add(tmpServerName.getServerName());
583 }
584 }
585
586 this.recoveringRegionLock.lock();
587 try {
588 List<String> tasks = ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
589 if (tasks != null) {
590 for (String t : tasks) {
591 byte[] data = ZKUtil.getData(this.watcher, ZKUtil.joinZNode(watcher.splitLogZNode, t));
592 if (data != null) {
593 SplitLogTask slt = null;
594 try {
595 slt = SplitLogTask.parseFrom(data);
596 } catch (DeserializationException e) {
597 LOG.warn("Failed parse data for znode " + t, e);
598 }
599 if (slt != null && slt.isDone()) {
600 continue;
601 }
602 }
603
604 t = ZKSplitLog.getFileName(t);
605 ServerName serverName = HLogUtil.getServerNameFromHLogDirectoryName(new Path(t));
606 if (serverName != null) {
607 knownFailedServers.add(serverName.getServerName());
608 } else {
609 LOG.warn("Found invalid WAL log file name:" + t);
610 }
611 }
612 }
613
614
615 List<String> regions = ZKUtil.listChildrenNoWatch(watcher, watcher.recoveringRegionsZNode);
616 if (regions != null) {
617 for (String region : regions) {
618 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, region);
619 List<String> regionFailedServers = ZKUtil.listChildrenNoWatch(watcher, nodePath);
620 if (regionFailedServers == null || regionFailedServers.isEmpty()) {
621 ZKUtil.deleteNode(watcher, nodePath);
622 continue;
623 }
624 boolean needMoreRecovery = false;
625 for (String tmpFailedServer : regionFailedServers) {
626 if (knownFailedServers.contains(tmpFailedServer)) {
627 needMoreRecovery = true;
628 break;
629 }
630 }
631 if (!needMoreRecovery) {
632 ZKUtil.deleteNodeRecursively(watcher, nodePath);
633 }
634 }
635 }
636 } finally {
637 this.recoveringRegionLock.unlock();
638 }
639 }
640
641 private void deleteRecoveringRegionZNodes(List<String> regions) {
642 try {
643 if (regions == null) {
644
645 LOG.info("Garbage collecting all recovering regions.");
646 ZKUtil.deleteChildrenRecursively(watcher, watcher.recoveringRegionsZNode);
647 } else {
648 for (String curRegion : regions) {
649 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, curRegion);
650 ZKUtil.deleteNodeRecursively(watcher, nodePath);
651 }
652 }
653 } catch (KeeperException e) {
654 LOG.warn("Cannot remove recovering regions from ZooKeeper", e);
655 }
656 }
657
658 private void setDone(String path, TerminationStatus status) {
659 Task task = tasks.get(path);
660 if (task == null) {
661 if (!ZKSplitLog.isRescanNode(watcher, path)) {
662 SplitLogCounters.tot_mgr_unacquired_orphan_done.incrementAndGet();
663 LOG.debug("unacquired orphan task is done " + path);
664 }
665 } else {
666 synchronized (task) {
667 if (task.status == IN_PROGRESS) {
668 if (status == SUCCESS) {
669 SplitLogCounters.tot_mgr_log_split_success.incrementAndGet();
670 LOG.info("Done splitting " + path);
671 } else {
672 SplitLogCounters.tot_mgr_log_split_err.incrementAndGet();
673 LOG.warn("Error splitting " + path);
674 }
675 task.status = status;
676 if (task.batch != null) {
677 synchronized (task.batch) {
678 if (status == SUCCESS) {
679 task.batch.done++;
680 } else {
681 task.batch.error++;
682 }
683 task.batch.notify();
684 }
685 }
686 }
687 }
688 }
689
690
691
692
693
694 deleteNode(path, zkretries);
695 return;
696 }
697
698 private void createNode(String path, Long retry_count) {
699 SplitLogTask slt = new SplitLogTask.Unassigned(serverName);
700 ZKUtil.asyncCreate(this.watcher, path, slt.toByteArray(), new CreateAsyncCallback(), retry_count);
701 SplitLogCounters.tot_mgr_node_create_queued.incrementAndGet();
702 return;
703 }
704
705 private void createNodeSuccess(String path) {
706 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
707 LOG.debug("put up splitlog task at znode " + path);
708 getDataSetWatch(path, zkretries);
709 }
710
711 private void createNodeFailure(String path) {
712
713 LOG.warn("failed to create task node" + path);
714 setDone(path, FAILURE);
715 }
716
717
718 private void getDataSetWatch(String path, Long retry_count) {
719 this.watcher.getRecoverableZooKeeper().getZooKeeper().
720 getData(path, this.watcher,
721 new GetDataAsyncCallback(), retry_count);
722 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
723 }
724
725 private void tryGetDataSetWatch(String path) {
726
727 this.watcher.getRecoverableZooKeeper().getZooKeeper().
728 getData(path, this.watcher,
729 new GetDataAsyncCallback(), Long.valueOf(-1)
730 SplitLogCounters.tot_mgr_get_data_queued.incrementAndGet();
731 }
732
733 private void getDataSetWatchSuccess(String path, byte[] data, int version)
734 throws DeserializationException {
735 if (data == null) {
736 if (version == Integer.MIN_VALUE) {
737
738 setDone(path, SUCCESS);
739 return;
740 }
741 SplitLogCounters.tot_mgr_null_data.incrementAndGet();
742 LOG.fatal("logic error - got null data " + path);
743 setDone(path, FAILURE);
744 return;
745 }
746 data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
747 SplitLogTask slt = SplitLogTask.parseFrom(data);
748 if (slt.isUnassigned()) {
749 LOG.debug("task not yet acquired " + path + " ver = " + version);
750 handleUnassignedTask(path);
751 } else if (slt.isOwned()) {
752 heartbeat(path, version, slt.getServerName());
753 } else if (slt.isResigned()) {
754 LOG.info("task " + path + " entered state: " + slt.toString());
755 resubmitOrFail(path, FORCE);
756 } else if (slt.isDone()) {
757 LOG.info("task " + path + " entered state: " + slt.toString());
758 if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
759 if (taskFinisher.finish(slt.getServerName(), ZKSplitLog.getFileName(path)) == Status.DONE) {
760 setDone(path, SUCCESS);
761 } else {
762 resubmitOrFail(path, CHECK);
763 }
764 } else {
765 setDone(path, SUCCESS);
766 }
767 } else if (slt.isErr()) {
768 LOG.info("task " + path + " entered state: " + slt.toString());
769 resubmitOrFail(path, CHECK);
770 } else {
771 LOG.fatal("logic error - unexpected zk state for path = " + path + " data = " + slt.toString());
772 setDone(path, FAILURE);
773 }
774 }
775
776 private void getDataSetWatchFailure(String path) {
777 LOG.warn("failed to set data watch " + path);
778 setDone(path, FAILURE);
779 }
780
781
782
783
784
785
786
787
788
789
790 private void handleUnassignedTask(String path) {
791 if (ZKSplitLog.isRescanNode(watcher, path)) {
792 return;
793 }
794 Task task = findOrCreateOrphanTask(path);
795 if (task.isOrphan() && (task.incarnation == 0)) {
796 LOG.info("resubmitting unassigned orphan task " + path);
797
798
799 resubmit(path, task, FORCE);
800 }
801 }
802
803
804
805
806
807
808
809 private boolean needAbandonRetries(int statusCode, String action) {
810 if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
811 LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
812 + "action=" + action);
813 return true;
814 }
815 return false;
816 }
817
818 private void heartbeat(String path, int new_version, ServerName workerName) {
819 Task task = findOrCreateOrphanTask(path);
820 if (new_version != task.last_version) {
821 if (task.isUnassigned()) {
822 LOG.info("task " + path + " acquired by " + workerName);
823 }
824 task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(), new_version, workerName);
825 SplitLogCounters.tot_mgr_heartbeat.incrementAndGet();
826 } else {
827
828
829
830
831 }
832 return;
833 }
834
835 private boolean resubmit(String path, Task task, ResubmitDirective directive) {
836
837 if (task.status != IN_PROGRESS) {
838 return false;
839 }
840 int version;
841 if (directive != FORCE) {
842
843
844
845
846
847 final long time = EnvironmentEdgeManager.currentTimeMillis() - task.last_update;
848 final boolean alive = master.getServerManager() != null ?
849 master.getServerManager().isServerOnline(task.cur_worker_name) : true;
850 if (alive && time < timeout) {
851 LOG.trace("Skipping the resubmit of " + task.toString() + " because the server " +
852 task.cur_worker_name + " is not marked as dead, we waited for " + time +
853 " while the timeout is " + timeout);
854 return false;
855 }
856 if (task.unforcedResubmits >= resubmit_threshold) {
857 if (!task.resubmitThresholdReached) {
858 task.resubmitThresholdReached = true;
859 SplitLogCounters.tot_mgr_resubmit_threshold_reached.incrementAndGet();
860 LOG.info("Skipping resubmissions of task " + path +
861 " because threshold " + resubmit_threshold + " reached");
862 }
863 return false;
864 }
865
866 version = task.last_version;
867 } else {
868 SplitLogCounters.tot_mgr_resubmit_force.incrementAndGet();
869 version = -1;
870 }
871 LOG.info("resubmitting task " + path);
872 task.incarnation++;
873 try {
874
875 SplitLogTask slt = new SplitLogTask.Unassigned(this.serverName);
876 if (ZKUtil.setData(this.watcher, path, slt.toByteArray(), version) == false) {
877 LOG.debug("failed to resubmit task " + path +
878 " version changed");
879 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
880 return false;
881 }
882 } catch (NoNodeException e) {
883 LOG.warn("failed to resubmit because znode doesn't exist " + path +
884 " task done (or forced done by removing the znode)");
885 try {
886 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
887 } catch (DeserializationException e1) {
888 LOG.debug("Failed to re-resubmit task " + path + " because of deserialization issue", e1);
889 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
890 return false;
891 }
892 return false;
893 } catch (KeeperException.BadVersionException e) {
894 LOG.debug("failed to resubmit task " + path + " version changed");
895 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
896 return false;
897 } catch (KeeperException e) {
898 SplitLogCounters.tot_mgr_resubmit_failed.incrementAndGet();
899 LOG.warn("failed to resubmit " + path, e);
900 return false;
901 }
902
903 if (directive != FORCE) {
904 task.unforcedResubmits++;
905 }
906 task.setUnassigned();
907 createRescanNode(Long.MAX_VALUE);
908 SplitLogCounters.tot_mgr_resubmit.incrementAndGet();
909 return true;
910 }
911
912 private void resubmitOrFail(String path, ResubmitDirective directive) {
913 if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
914 setDone(path, FAILURE);
915 }
916 }
917
918 private void deleteNode(String path, Long retries) {
919 SplitLogCounters.tot_mgr_node_delete_queued.incrementAndGet();
920
921
922
923 this.watcher.getRecoverableZooKeeper().getZooKeeper().
924 delete(path, -1, new DeleteAsyncCallback(),
925 retries);
926 }
927
928 private void deleteNodeSuccess(String path) {
929 if (ignoreZKDeleteForTesting) {
930 return;
931 }
932 Task task;
933 task = tasks.remove(path);
934 if (task == null) {
935 if (ZKSplitLog.isRescanNode(watcher, path)) {
936 SplitLogCounters.tot_mgr_rescan_deleted.incrementAndGet();
937 }
938 SplitLogCounters.tot_mgr_missing_state_in_delete.incrementAndGet();
939 LOG.debug("deleted task without in memory state " + path);
940 return;
941 }
942 synchronized (task) {
943 task.status = DELETED;
944 task.notify();
945 }
946 SplitLogCounters.tot_mgr_task_deleted.incrementAndGet();
947 }
948
949 private void deleteNodeFailure(String path) {
950 LOG.info("Failed to delete node " + path + " and will retry soon.");
951 return;
952 }
953
954
955
956
957
958
959 private void createRescanNode(long retries) {
960
961
962
963
964
965
966
967 SplitLogTask slt = new SplitLogTask.Done(this.serverName);
968 this.watcher.getRecoverableZooKeeper().getZooKeeper().
969 create(ZKSplitLog.getRescanNode(watcher), slt.toByteArray(),
970 Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL,
971 new CreateRescanAsyncCallback(), Long.valueOf(retries));
972 }
973
974 private void createRescanSuccess(String path) {
975 lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
976 SplitLogCounters.tot_mgr_rescan.incrementAndGet();
977 getDataSetWatch(path, zkretries);
978 }
979
980 private void createRescanFailure() {
981 LOG.fatal("logic failure, rescan failure must not happen");
982 }
983
984
985
986
987
988
989 private Task createTaskIfAbsent(String path, TaskBatch batch) {
990 Task oldtask;
991
992
993 Task newtask = new Task();
994 newtask.batch = batch;
995 oldtask = tasks.putIfAbsent(path, newtask);
996 if (oldtask == null) {
997 batch.installed++;
998 return null;
999 }
1000
1001 synchronized (oldtask) {
1002 if (oldtask.isOrphan()) {
1003 if (oldtask.status == SUCCESS) {
1004
1005
1006
1007
1008 return (null);
1009 }
1010 if (oldtask.status == IN_PROGRESS) {
1011 oldtask.batch = batch;
1012 batch.installed++;
1013 LOG.debug("Previously orphan task " + path + " is now being waited upon");
1014 return null;
1015 }
1016 while (oldtask.status == FAILURE) {
1017 LOG.debug("wait for status of task " + path + " to change to DELETED");
1018 SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
1019 try {
1020 oldtask.wait();
1021 } catch (InterruptedException e) {
1022 Thread.currentThread().interrupt();
1023 LOG.warn("Interrupted when waiting for znode delete callback");
1024
1025 break;
1026 }
1027 }
1028 if (oldtask.status != DELETED) {
1029 LOG.warn("Failure because previously failed task" +
1030 " state still present. Waiting for znode delete callback" +
1031 " path=" + path);
1032 return oldtask;
1033 }
1034
1035 Task t = tasks.putIfAbsent(path, newtask);
1036 if (t == null) {
1037 batch.installed++;
1038 return null;
1039 }
1040 LOG.fatal("Logic error. Deleted task still present in tasks map");
1041 assert false : "Deleted task still present in tasks map";
1042 return t;
1043 }
1044 LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
1045 return oldtask;
1046 }
1047 }
1048
1049 Task findOrCreateOrphanTask(String path) {
1050 Task orphanTask = new Task();
1051 Task task;
1052 task = tasks.putIfAbsent(path, orphanTask);
1053 if (task == null) {
1054 LOG.info("creating orphan task " + path);
1055 SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
1056 task = orphanTask;
1057 }
1058 return task;
1059 }
1060
1061 @Override
1062 public void nodeDataChanged(String path) {
1063 Task task;
1064 task = tasks.get(path);
1065 if (task != null || ZKSplitLog.isRescanNode(watcher, path)) {
1066 if (task != null) {
1067 task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
1068 }
1069 getDataSetWatch(path, zkretries);
1070 }
1071 }
1072
1073 public void stop() {
1074 if (timeoutMonitor != null) {
1075 timeoutMonitor.interrupt();
1076 }
1077 }
1078
1079 private void lookForOrphans() {
1080 List<String> orphans;
1081 try {
1082 orphans = ZKUtil.listChildrenNoWatch(this.watcher,
1083 this.watcher.splitLogZNode);
1084 if (orphans == null) {
1085 LOG.warn("could not get children of " + this.watcher.splitLogZNode);
1086 return;
1087 }
1088 } catch (KeeperException e) {
1089 LOG.warn("could not get children of " + this.watcher.splitLogZNode +
1090 " " + StringUtils.stringifyException(e));
1091 return;
1092 }
1093 int rescan_nodes = 0;
1094 for (String path : orphans) {
1095 String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
1096 if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
1097 rescan_nodes++;
1098 LOG.debug("found orphan rescan node " + path);
1099 } else {
1100 LOG.info("found orphan task " + path);
1101 }
1102 getDataSetWatch(nodepath, zkretries);
1103 }
1104 LOG.info("Found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
1105 rescan_nodes + " rescan nodes");
1106 }
1107
1108
1109
1110
1111
1112
1113
1114 void markRegionsRecoveringInZK(final ServerName serverName, Set<HRegionInfo> userRegions)
1115 throws KeeperException {
1116 if (userRegions == null || !this.distributedLogReplay) {
1117 return;
1118 }
1119
1120 try {
1121 this.recoveringRegionLock.lock();
1122
1123 this.lastRecoveringNodeCreationTime = EnvironmentEdgeManager.currentTimeMillis();
1124
1125 for (HRegionInfo region : userRegions) {
1126 String regionEncodeName = region.getEncodedName();
1127 long retries = this.zkretries;
1128
1129 do {
1130 String nodePath = ZKUtil.joinZNode(watcher.recoveringRegionsZNode, regionEncodeName);
1131 long lastRecordedFlushedSequenceId = -1;
1132 try {
1133 long lastSequenceId = this.master.getServerManager().getLastFlushedSequenceId(
1134 regionEncodeName.getBytes());
1135
1136
1137
1138
1139
1140 byte[] data = ZKUtil.getData(this.watcher, nodePath);
1141 if (data == null) {
1142 ZKUtil.createSetData(this.watcher, nodePath,
1143 ZKUtil.positionToByteArray(lastSequenceId));
1144 } else {
1145 lastRecordedFlushedSequenceId = SplitLogManager.parseLastFlushedSequenceIdFrom(data);
1146 if (lastRecordedFlushedSequenceId < lastSequenceId) {
1147
1148 ZKUtil.setData(this.watcher, nodePath, ZKUtil.positionToByteArray(lastSequenceId));
1149 }
1150 }
1151
1152 nodePath = ZKUtil.joinZNode(nodePath, serverName.getServerName());
1153 if (lastSequenceId <= lastRecordedFlushedSequenceId) {
1154
1155 lastSequenceId = lastRecordedFlushedSequenceId;
1156 }
1157 ZKUtil.createSetData(this.watcher, nodePath,
1158 ZKUtil.regionSequenceIdsToByteArray(lastSequenceId, null));
1159 LOG.debug("Mark region " + regionEncodeName + " recovering from failed region server "
1160 + serverName);
1161
1162
1163 break;
1164 } catch (KeeperException e) {
1165
1166 if (retries <= 1) {
1167 throw e;
1168 }
1169
1170 try {
1171 Thread.sleep(20);
1172 } catch (Exception ignoreE) {
1173
1174 }
1175 }
1176 } while ((--retries) > 0 && (!this.stopper.isStopped()));
1177 }
1178 } finally {
1179 this.recoveringRegionLock.unlock();
1180 }
1181 }
1182
1183
1184
1185
1186
1187 public static long parseLastFlushedSequenceIdFrom(final byte[] bytes) {
1188 long lastRecordedFlushedSequenceId = -1l;
1189 try {
1190 lastRecordedFlushedSequenceId = ZKUtil.parseHLogPositionFrom(bytes);
1191 } catch (DeserializationException e) {
1192 lastRecordedFlushedSequenceId = -1l;
1193 LOG.warn("Can't parse last flushed sequence Id", e);
1194 }
1195 return lastRecordedFlushedSequenceId;
1196 }
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206 public static boolean
1207 isRegionMarkedRecoveringInZK(ZooKeeperWatcher zkw, String regionEncodedName)
1208 throws KeeperException {
1209 boolean result = false;
1210 String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, regionEncodedName);
1211
1212 byte[] node = ZKUtil.getDataAndWatch(zkw, nodePath);
1213 if (node != null) {
1214 result = true;
1215 }
1216 return result;
1217 }
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227 public static RegionStoreSequenceIds getRegionFlushedSequenceId(ZooKeeperWatcher zkw,
1228 String serverName, String encodedRegionName) throws IOException {
1229
1230
1231
1232
1233
1234
1235
1236 RegionStoreSequenceIds result = null;
1237 String nodePath = ZKUtil.joinZNode(zkw.recoveringRegionsZNode, encodedRegionName);
1238 nodePath = ZKUtil.joinZNode(nodePath, serverName);
1239 try {
1240 byte[] data = ZKUtil.getData(zkw, nodePath);
1241 if (data != null) {
1242 result = ZKUtil.parseRegionStoreSequenceIds(data);
1243 }
1244 } catch (KeeperException e) {
1245 throw new IOException("Cannot get lastFlushedSequenceId from ZooKeeper for server="
1246 + serverName + "; region=" + encodedRegionName, e);
1247 } catch (DeserializationException e) {
1248 LOG.warn("Can't parse last flushed sequence Id from znode:" + nodePath, e);
1249 }
1250 return result;
1251 }
1252
1253
1254
1255
1256
1257
1258
1259 static class TaskBatch {
1260 int installed = 0;
1261 int done = 0;
1262 int error = 0;
1263 volatile boolean isDead = false;
1264
1265 @Override
1266 public String toString() {
1267 return ("installed = " + installed + " done = " + done + " error = " + error);
1268 }
1269 }
1270
1271
1272
1273
1274 static class Task {
1275 volatile long last_update;
1276 volatile int last_version;
1277 volatile ServerName cur_worker_name;
1278 volatile TaskBatch batch;
1279 volatile TerminationStatus status;
1280 volatile int incarnation;
1281 volatile int unforcedResubmits;
1282 volatile boolean resubmitThresholdReached;
1283
1284 @Override
1285 public String toString() {
1286 return ("last_update = " + last_update +
1287 " last_version = " + last_version +
1288 " cur_worker_name = " + cur_worker_name +
1289 " status = " + status +
1290 " incarnation = " + incarnation +
1291 " resubmits = " + unforcedResubmits +
1292 " batch = " + batch);
1293 }
1294
1295 Task() {
1296 incarnation = 0;
1297 last_version = -1;
1298 status = IN_PROGRESS;
1299 setUnassigned();
1300 }
1301
1302 public boolean isOrphan() {
1303 return (batch == null || batch.isDead);
1304 }
1305
1306 public boolean isUnassigned() {
1307 return (cur_worker_name == null);
1308 }
1309
1310 public void heartbeatNoDetails(long time) {
1311 last_update = time;
1312 }
1313
1314 public void heartbeat(long time, int version, ServerName worker) {
1315 last_version = version;
1316 last_update = time;
1317 cur_worker_name = worker;
1318 }
1319
1320 public void setUnassigned() {
1321 cur_worker_name = null;
1322 last_update = -1;
1323 }
1324 }
1325
1326 void handleDeadWorker(ServerName workerName) {
1327
1328
1329 synchronized (deadWorkersLock) {
1330 if (deadWorkers == null) {
1331 deadWorkers = new HashSet<ServerName>(100);
1332 }
1333 deadWorkers.add(workerName);
1334 }
1335 LOG.info("dead splitlog worker " + workerName);
1336 }
1337
1338 void handleDeadWorkers(Set<ServerName> serverNames) {
1339 synchronized (deadWorkersLock) {
1340 if (deadWorkers == null) {
1341 deadWorkers = new HashSet<ServerName>(100);
1342 }
1343 deadWorkers.addAll(serverNames);
1344 }
1345 LOG.info("dead splitlog workers " + serverNames);
1346 }
1347
1348
1349
1350
1351
1352 private class TimeoutMonitor extends Chore {
1353 private long lastLog = 0;
1354
1355 public TimeoutMonitor(final int period, Stoppable stopper) {
1356 super("SplitLogManager Timeout Monitor", period, stopper);
1357 }
1358
1359 @Override
1360 protected void chore() {
1361 int resubmitted = 0;
1362 int unassigned = 0;
1363 int tot = 0;
1364 boolean found_assigned_task = false;
1365 Set<ServerName> localDeadWorkers;
1366
1367 synchronized (deadWorkersLock) {
1368 localDeadWorkers = deadWorkers;
1369 deadWorkers = null;
1370 }
1371
1372 for (Map.Entry<String, Task> e : tasks.entrySet()) {
1373 String path = e.getKey();
1374 Task task = e.getValue();
1375 ServerName cur_worker = task.cur_worker_name;
1376 tot++;
1377
1378
1379
1380
1381
1382 if (task.isUnassigned()) {
1383 unassigned++;
1384 continue;
1385 }
1386 found_assigned_task = true;
1387 if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
1388 SplitLogCounters.tot_mgr_resubmit_dead_server_task.incrementAndGet();
1389 if (resubmit(path, task, FORCE)) {
1390 resubmitted++;
1391 } else {
1392 handleDeadWorker(cur_worker);
1393 LOG.warn("Failed to resubmit task " + path + " owned by dead " +
1394 cur_worker + ", will retry.");
1395 }
1396 } else if (resubmit(path, task, CHECK)) {
1397 resubmitted++;
1398 }
1399 }
1400 if (tot > 0) {
1401 long now = EnvironmentEdgeManager.currentTimeMillis();
1402 if (now > lastLog + 5000) {
1403 lastLog = now;
1404 LOG.info("total tasks = " + tot + " unassigned = " + unassigned + " tasks=" + tasks);
1405 }
1406 }
1407 if (resubmitted > 0) {
1408 LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
1409 }
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419 if (tot > 0 && !found_assigned_task &&
1420 ((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
1421 unassignedTimeout)) {
1422 for (Map.Entry<String, Task> e : tasks.entrySet()) {
1423 String path = e.getKey();
1424 Task task = e.getValue();
1425
1426
1427
1428
1429 if (task.isUnassigned() && (task.status != FAILURE)) {
1430
1431 tryGetDataSetWatch(path);
1432 }
1433 }
1434 createRescanNode(Long.MAX_VALUE);
1435 SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
1436 LOG.debug("resubmitting unassigned task(s) after timeout");
1437 }
1438
1439
1440 if (failedDeletions.size() > 0) {
1441 List<String> tmpPaths = new ArrayList<String>(failedDeletions);
1442 for (String tmpPath : tmpPaths) {
1443
1444 deleteNode(tmpPath, zkretries);
1445 }
1446 failedDeletions.removeAll(tmpPaths);
1447 }
1448
1449
1450 long timeInterval = EnvironmentEdgeManager.currentTimeMillis()
1451 - lastRecoveringNodeCreationTime;
1452 if (!failedRecoveringRegionDeletions.isEmpty()
1453 || (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) {
1454
1455 if (!failedRecoveringRegionDeletions.isEmpty()) {
1456 List<Pair<Set<ServerName>, Boolean>> previouslyFailedDeletions =
1457 new ArrayList<Pair<Set<ServerName>, Boolean>>(failedRecoveringRegionDeletions);
1458 failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletions);
1459 for (Pair<Set<ServerName>, Boolean> failedDeletion : previouslyFailedDeletions) {
1460 removeRecoveringRegionsFromZK(failedDeletion.getFirst(), failedDeletion.getSecond());
1461 }
1462 } else {
1463 removeRecoveringRegionsFromZK(null, null);
1464 }
1465 }
1466 }
1467 }
1468
1469
1470
1471
1472
1473 class CreateAsyncCallback implements AsyncCallback.StringCallback {
1474 private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
1475
1476 @Override
1477 public void processResult(int rc, String path, Object ctx, String name) {
1478 SplitLogCounters.tot_mgr_node_create_result.incrementAndGet();
1479 if (rc != 0) {
1480 if (needAbandonRetries(rc, "Create znode " + path)) {
1481 createNodeFailure(path);
1482 return;
1483 }
1484 if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
1485
1486
1487
1488
1489
1490
1491 LOG.debug("found pre-existing znode " + path);
1492 SplitLogCounters.tot_mgr_node_already_exists.incrementAndGet();
1493 } else {
1494 Long retry_count = (Long)ctx;
1495 LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
1496 path + " remaining retries=" + retry_count);
1497 if (retry_count == 0) {
1498 SplitLogCounters.tot_mgr_node_create_err.incrementAndGet();
1499 createNodeFailure(path);
1500 } else {
1501 SplitLogCounters.tot_mgr_node_create_retry.incrementAndGet();
1502 createNode(path, retry_count - 1);
1503 }
1504 return;
1505 }
1506 }
1507 createNodeSuccess(path);
1508 }
1509 }
1510
1511
1512
1513
1514
1515 class GetDataAsyncCallback implements AsyncCallback.DataCallback {
1516 private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
1517
1518 @Override
1519 public void processResult(int rc, String path, Object ctx, byte[] data,
1520 Stat stat) {
1521 SplitLogCounters.tot_mgr_get_data_result.incrementAndGet();
1522 if (rc != 0) {
1523 if (needAbandonRetries(rc, "GetData from znode " + path)) {
1524 return;
1525 }
1526 if (rc == KeeperException.Code.NONODE.intValue()) {
1527 SplitLogCounters.tot_mgr_get_data_nonode.incrementAndGet();
1528
1529
1530
1531 LOG.warn("task znode " + path + " vanished.");
1532 try {
1533 getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
1534 } catch (DeserializationException e) {
1535 LOG.warn("Deserialization problem", e);
1536 }
1537 return;
1538 }
1539 Long retry_count = (Long) ctx;
1540
1541 if (retry_count < 0) {
1542 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1543 path + ". Ignoring error. No error handling. No retrying.");
1544 return;
1545 }
1546 LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1547 path + " remaining retries=" + retry_count);
1548 if (retry_count == 0) {
1549 SplitLogCounters.tot_mgr_get_data_err.incrementAndGet();
1550 getDataSetWatchFailure(path);
1551 } else {
1552 SplitLogCounters.tot_mgr_get_data_retry.incrementAndGet();
1553 getDataSetWatch(path, retry_count - 1);
1554 }
1555 return;
1556 }
1557 try {
1558 getDataSetWatchSuccess(path, data, stat.getVersion());
1559 } catch (DeserializationException e) {
1560 LOG.warn("Deserialization problem", e);
1561 }
1562 return;
1563 }
1564 }
1565
1566
1567
1568
1569
1570 class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
1571 private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
1572
1573 @Override
1574 public void processResult(int rc, String path, Object ctx) {
1575 SplitLogCounters.tot_mgr_node_delete_result.incrementAndGet();
1576 if (rc != 0) {
1577 if (needAbandonRetries(rc, "Delete znode " + path)) {
1578 failedDeletions.add(path);
1579 return;
1580 }
1581 if (rc != KeeperException.Code.NONODE.intValue()) {
1582 SplitLogCounters.tot_mgr_node_delete_err.incrementAndGet();
1583 Long retry_count = (Long) ctx;
1584 LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
1585 path + " remaining retries=" + retry_count);
1586 if (retry_count == 0) {
1587 LOG.warn("delete failed " + path);
1588 failedDeletions.add(path);
1589 deleteNodeFailure(path);
1590 } else {
1591 deleteNode(path, retry_count - 1);
1592 }
1593 return;
1594 } else {
1595 LOG.info(path +
1596 " does not exist. Either was created but deleted behind our" +
1597 " back by another pending delete OR was deleted" +
1598 " in earlier retry rounds. zkretries = " + (Long) ctx);
1599 }
1600 } else {
1601 LOG.debug("deleted " + path);
1602 }
1603 deleteNodeSuccess(path);
1604 }
1605 }
1606
1607
1608
1609
1610
1611
1612
1613
1614 class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
1615 private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
1616
1617 @Override
1618 public void processResult(int rc, String path, Object ctx, String name) {
1619 if (rc != 0) {
1620 if (needAbandonRetries(rc, "CreateRescan znode " + path)) {
1621 return;
1622 }
1623 Long retry_count = (Long)ctx;
1624 LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
1625 " remaining retries=" + retry_count);
1626 if (retry_count == 0) {
1627 createRescanFailure();
1628 } else {
1629 createRescanNode(retry_count - 1);
1630 }
1631 return;
1632 }
1633
1634 createRescanSuccess(name);
1635 }
1636 }
1637
1638
1639
1640
1641
1642
1643
1644 public interface TaskFinisher {
1645
1646
1647
1648 enum Status {
1649
1650
1651
1652 DONE(),
1653
1654
1655
1656 ERR();
1657 }
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667 Status finish(ServerName workerName, String taskname);
1668 }
1669
1670 enum ResubmitDirective {
1671 CHECK(),
1672 FORCE();
1673 }
1674
1675 enum TerminationStatus {
1676 IN_PROGRESS("in_progress"),
1677 SUCCESS("success"),
1678 FAILURE("failure"),
1679 DELETED("deleted");
1680
1681 String statusMsg;
1682 TerminationStatus(String msg) {
1683 statusMsg = msg;
1684 }
1685
1686 @Override
1687 public String toString() {
1688 return statusMsg;
1689 }
1690 }
1691 }