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