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.hbase.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.HRegionInfo;
50 import org.apache.hadoop.hbase.Server;
51 import org.apache.hadoop.hbase.ServerName;
52 import org.apache.hadoop.hbase.SplitLogCounters;
53 import org.apache.hadoop.hbase.Stoppable;
54 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
55 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
56 import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
57 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
58 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
59 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
60 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
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
66 import com.google.common.annotations.VisibleForTesting;
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97 @InterfaceAudience.Private
98 public class SplitLogManager {
99 private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
100
101 private Server server;
102
103 private final Stoppable stopper;
104 private final Configuration conf;
105
106 public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000);
107
108 private long unassignedTimeout;
109 private long lastTaskCreateTime = Long.MAX_VALUE;
110 private long checkRecoveringTimeThreshold = 15000;
111 private final List<Pair<Set<ServerName>, Boolean>> failedRecoveringRegionDeletions = Collections
112 .synchronizedList(new ArrayList<Pair<Set<ServerName>, Boolean>>());
113
114
115
116
117
118 protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
119
120 private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
121 private TimeoutMonitor timeoutMonitor;
122
123 private volatile Set<ServerName> deadWorkers = null;
124 private final Object deadWorkersLock = new Object();
125
126
127
128
129
130
131
132
133
134
135
136 public SplitLogManager(Server server, Configuration conf, Stoppable stopper,
137 MasterServices master, ServerName serverName) throws IOException {
138 this.server = server;
139 this.conf = conf;
140 this.stopper = stopper;
141 if (server.getCoordinatedStateManager() != null) {
142 SplitLogManagerCoordination coordination =
143 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
144 .getSplitLogManagerCoordination();
145 Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());
146 SplitLogManagerDetails details =
147 new SplitLogManagerDetails(tasks, master, failedDeletions, serverName);
148 coordination.init();
149 coordination.setDetails(details);
150
151 }
152 this.unassignedTimeout =
153 conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
154 this.timeoutMonitor =
155 new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000),
156 stopper);
157 Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
158 + ".splitLogManagerTimeoutMonitor");
159 }
160
161 private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
162 return getFileList(conf, logDirs, filter);
163 }
164
165
166
167
168
169
170
171
172
173
174
175
176 @VisibleForTesting
177 public static FileStatus[] getFileList(final Configuration conf, final List<Path> logDirs,
178 final PathFilter filter)
179 throws IOException {
180 List<FileStatus> fileStatus = new ArrayList<FileStatus>();
181 for (Path logDir : logDirs) {
182 final FileSystem fs = logDir.getFileSystem(conf);
183 if (!fs.exists(logDir)) {
184 LOG.warn(logDir + " doesn't exist. Nothing to do!");
185 continue;
186 }
187 FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter);
188 if (logfiles == null || logfiles.length == 0) {
189 LOG.info(logDir + " is empty dir, no logs to split");
190 } else {
191 Collections.addAll(fileStatus, logfiles);
192 }
193 }
194 FileStatus[] a = new FileStatus[fileStatus.size()];
195 return fileStatus.toArray(a);
196 }
197
198
199
200
201
202
203
204 public long splitLogDistributed(final Path logDir) throws IOException {
205 List<Path> logDirs = new ArrayList<Path>();
206 logDirs.add(logDir);
207 return splitLogDistributed(logDirs);
208 }
209
210
211
212
213
214
215
216
217
218 public long splitLogDistributed(final List<Path> logDirs) throws IOException {
219 if (logDirs.isEmpty()) {
220 return 0;
221 }
222 Set<ServerName> serverNames = new HashSet<ServerName>();
223 for (Path logDir : logDirs) {
224 try {
225 ServerName serverName = DefaultWALProvider.getServerNameFromWALDirectoryName(logDir);
226 if (serverName != null) {
227 serverNames.add(serverName);
228 }
229 } catch (IllegalArgumentException e) {
230
231 LOG.warn("Cannot parse server name from " + logDir);
232 }
233 }
234 return splitLogDistributed(serverNames, logDirs, null);
235 }
236
237
238
239
240
241
242
243
244
245
246 public long splitLogDistributed(final Set<ServerName> serverNames, final List<Path> logDirs,
247 PathFilter filter) throws IOException {
248 MonitoredTask status = TaskMonitor.get().createStatus("Doing distributed log split in " +
249 logDirs + " for serverName=" + serverNames);
250 FileStatus[] logfiles = getFileList(logDirs, filter);
251 status.setStatus("Checking directory contents...");
252 LOG.debug("Scheduling batch of logs to split");
253 SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
254 LOG.info("started splitting " + logfiles.length + " logs in " + logDirs +
255 " for " + serverNames);
256 long t = EnvironmentEdgeManager.currentTime();
257 long totalSize = 0;
258 TaskBatch batch = new TaskBatch();
259 Boolean isMetaRecovery = (filter == null) ? null : false;
260 for (FileStatus lf : logfiles) {
261
262
263
264
265
266 totalSize += lf.getLen();
267 String pathToLog = FSUtils.removeRootPath(lf.getPath(), conf);
268 if (!enqueueSplitTask(pathToLog, batch)) {
269 throw new IOException("duplicate log split scheduled for " + lf.getPath());
270 }
271 }
272 waitForSplittingCompletion(batch, status);
273
274 if (filter == MasterFileSystem.META_FILTER
275
276
277 isMetaRecovery = true;
278 }
279 removeRecoveringRegions(serverNames, isMetaRecovery);
280
281 if (batch.done != batch.installed) {
282 batch.isDead = true;
283 SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
284 LOG.warn("error while splitting logs in " + logDirs + " installed = " + batch.installed
285 + " but only " + batch.done + " done");
286 String msg = "error or interrupted while splitting logs in " + logDirs + " Task = " + batch;
287 status.abort(msg);
288 throw new IOException(msg);
289 }
290 for (Path logDir : logDirs) {
291 status.setStatus("Cleaning up log directory...");
292 final FileSystem fs = logDir.getFileSystem(conf);
293 try {
294 if (fs.exists(logDir) && !fs.delete(logDir, false)) {
295 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
296 }
297 } catch (IOException ioe) {
298 FileStatus[] files = fs.listStatus(logDir);
299 if (files != null && files.length > 0) {
300 LOG.warn("returning success without actually splitting and "
301 + "deleting all the log files in path " + logDir);
302 } else {
303 LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
304 }
305 }
306 SplitLogCounters.tot_mgr_log_split_batch_success.incrementAndGet();
307 }
308 String msg =
309 "finished splitting (more than or equal to) " + totalSize + " bytes in " + batch.installed
310 + " log files in " + logDirs + " in "
311 + (EnvironmentEdgeManager.currentTime() - t) + "ms";
312 status.markComplete(msg);
313 LOG.info(msg);
314 return totalSize;
315 }
316
317
318
319
320
321
322
323 boolean enqueueSplitTask(String taskname, TaskBatch batch) {
324 lastTaskCreateTime = EnvironmentEdgeManager.currentTime();
325 String task =
326 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
327 .getSplitLogManagerCoordination().prepareTask(taskname);
328 Task oldtask = createTaskIfAbsent(task, batch);
329 if (oldtask == null) {
330
331 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
332 .getSplitLogManagerCoordination().submitTask(task);
333 return true;
334 }
335 return false;
336 }
337
338 private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {
339 synchronized (batch) {
340 while ((batch.done + batch.error) != batch.installed) {
341 try {
342 status.setStatus("Waiting for distributed tasks to finish. " + " scheduled="
343 + batch.installed + " done=" + batch.done + " error=" + batch.error);
344 int remaining = batch.installed - (batch.done + batch.error);
345 int actual = activeTasks(batch);
346 if (remaining != actual) {
347 LOG.warn("Expected " + remaining + " active tasks, but actually there are " + actual);
348 }
349 int remainingTasks =
350 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
351 .getSplitLogManagerCoordination().remainingTasksInCoordination();
352 if (remainingTasks >= 0 && actual > remainingTasks) {
353 LOG.warn("Expected at least" + actual + " tasks remaining, but actually there are "
354 + remainingTasks);
355 }
356 if (remainingTasks == 0 || actual == 0) {
357 LOG.warn("No more task remaining, splitting "
358 + "should have completed. Remaining tasks is " + remainingTasks
359 + ", active tasks in map " + actual);
360 if (remainingTasks == 0 && actual == 0) {
361 return;
362 }
363 }
364 batch.wait(100);
365 if (stopper.isStopped()) {
366 LOG.warn("Stopped while waiting for log splits to be completed");
367 return;
368 }
369 } catch (InterruptedException e) {
370 LOG.warn("Interrupted while waiting for log splits to be completed");
371 Thread.currentThread().interrupt();
372 return;
373 }
374 }
375 }
376 }
377
378 @VisibleForTesting
379 ConcurrentMap<String, Task> getTasks() {
380 return tasks;
381 }
382
383 private int activeTasks(final TaskBatch batch) {
384 int count = 0;
385 for (Task t : tasks.values()) {
386 if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
387 count++;
388 }
389 }
390 return count;
391
392 }
393
394
395
396
397
398
399
400
401 private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
402 if (!isLogReplaying()) {
403
404 return;
405 }
406
407 Set<String> recoveredServerNameSet = new HashSet<String>();
408 if (serverNames != null) {
409 for (ServerName tmpServerName : serverNames) {
410 recoveredServerNameSet.add(tmpServerName.getServerName());
411 }
412 }
413
414 try {
415 this.recoveringRegionLock.lock();
416 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
417 .getSplitLogManagerCoordination().removeRecoveringRegions(recoveredServerNameSet,
418 isMetaRecovery);
419 } catch (IOException e) {
420 LOG.warn("removeRecoveringRegions got exception. Will retry", e);
421 if (serverNames != null && !serverNames.isEmpty()) {
422 this.failedRecoveringRegionDeletions.add(new Pair<Set<ServerName>, Boolean>(serverNames,
423 isMetaRecovery));
424 }
425 } finally {
426 this.recoveringRegionLock.unlock();
427 }
428 }
429
430
431
432
433
434
435
436 void removeStaleRecoveringRegions(final Set<ServerName> failedServers) throws IOException,
437 InterruptedIOException {
438 Set<String> knownFailedServers = new HashSet<String>();
439 if (failedServers != null) {
440 for (ServerName tmpServerName : failedServers) {
441 knownFailedServers.add(tmpServerName.getServerName());
442 }
443 }
444
445 this.recoveringRegionLock.lock();
446 try {
447 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
448 .getSplitLogManagerCoordination().removeStaleRecoveringRegions(knownFailedServers);
449 } finally {
450 this.recoveringRegionLock.unlock();
451 }
452 }
453
454
455
456
457
458
459 private Task createTaskIfAbsent(String path, TaskBatch batch) {
460 Task oldtask;
461
462
463 Task newtask = new Task();
464 newtask.batch = batch;
465 oldtask = tasks.putIfAbsent(path, newtask);
466 if (oldtask == null) {
467 batch.installed++;
468 return null;
469 }
470
471 synchronized (oldtask) {
472 if (oldtask.isOrphan()) {
473 if (oldtask.status == SUCCESS) {
474
475
476
477
478 return (null);
479 }
480 if (oldtask.status == IN_PROGRESS) {
481 oldtask.batch = batch;
482 batch.installed++;
483 LOG.debug("Previously orphan task " + path + " is now being waited upon");
484 return null;
485 }
486 while (oldtask.status == FAILURE) {
487 LOG.debug("wait for status of task " + path + " to change to DELETED");
488 SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
489 try {
490 oldtask.wait();
491 } catch (InterruptedException e) {
492 Thread.currentThread().interrupt();
493 LOG.warn("Interrupted when waiting for znode delete callback");
494
495 break;
496 }
497 }
498 if (oldtask.status != DELETED) {
499 LOG.warn("Failure because previously failed task"
500 + " state still present. Waiting for znode delete callback" + " path=" + path);
501 return oldtask;
502 }
503
504 Task t = tasks.putIfAbsent(path, newtask);
505 if (t == null) {
506 batch.installed++;
507 return null;
508 }
509 LOG.fatal("Logic error. Deleted task still present in tasks map");
510 assert false : "Deleted task still present in tasks map";
511 return t;
512 }
513 LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
514 return oldtask;
515 }
516 }
517
518 Task findOrCreateOrphanTask(String path) {
519 Task orphanTask = new Task();
520 Task task;
521 task = tasks.putIfAbsent(path, orphanTask);
522 if (task == null) {
523 LOG.info("creating orphan task " + path);
524 SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
525 task = orphanTask;
526 }
527 return task;
528 }
529
530 public void stop() {
531 if (timeoutMonitor != null) {
532 timeoutMonitor.interrupt();
533 }
534 }
535
536 void handleDeadWorker(ServerName workerName) {
537
538
539 synchronized (deadWorkersLock) {
540 if (deadWorkers == null) {
541 deadWorkers = new HashSet<ServerName>(100);
542 }
543 deadWorkers.add(workerName);
544 }
545 LOG.info("dead splitlog worker " + workerName);
546 }
547
548 void handleDeadWorkers(Set<ServerName> serverNames) {
549 synchronized (deadWorkersLock) {
550 if (deadWorkers == null) {
551 deadWorkers = new HashSet<ServerName>(100);
552 }
553 deadWorkers.addAll(serverNames);
554 }
555 LOG.info("dead splitlog workers " + serverNames);
556 }
557
558
559
560
561
562
563
564 public void setRecoveryMode(boolean isForInitialization) throws IOException {
565 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
566 .getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
567
568 }
569
570 public void markRegionsRecovering(ServerName server, Set<HRegionInfo> userRegions)
571 throws InterruptedIOException, IOException {
572 if (userRegions == null || (!isLogReplaying())) {
573 return;
574 }
575 try {
576 this.recoveringRegionLock.lock();
577
578 ((BaseCoordinatedStateManager) this.server.getCoordinatedStateManager())
579 .getSplitLogManagerCoordination().markRegionsRecovering(server, userRegions);
580 } finally {
581 this.recoveringRegionLock.unlock();
582 }
583
584 }
585
586
587
588
589 public boolean isLogReplaying() {
590 if (server.getCoordinatedStateManager() == null) return false;
591 return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
592 .getSplitLogManagerCoordination().isReplaying();
593 }
594
595
596
597
598 public boolean isLogSplitting() {
599 if (server.getCoordinatedStateManager() == null) return false;
600 return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
601 .getSplitLogManagerCoordination().isSplitting();
602 }
603
604
605
606
607 public RecoveryMode getRecoveryMode() {
608 return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
609 .getSplitLogManagerCoordination().getRecoveryMode();
610 }
611
612
613
614
615
616
617
618 @InterfaceAudience.Private
619 public static class TaskBatch {
620 public int installed = 0;
621 public int done = 0;
622 public int error = 0;
623 public volatile boolean isDead = false;
624
625 @Override
626 public String toString() {
627 return ("installed = " + installed + " done = " + done + " error = " + error);
628 }
629 }
630
631
632
633
634 @InterfaceAudience.Private
635 public static class Task {
636 public volatile long last_update;
637 public volatile int last_version;
638 public volatile ServerName cur_worker_name;
639 public volatile TaskBatch batch;
640 public volatile TerminationStatus status;
641 public volatile int incarnation;
642 public final AtomicInteger unforcedResubmits = new AtomicInteger();
643 public volatile boolean resubmitThresholdReached;
644
645 @Override
646 public String toString() {
647 return ("last_update = " + last_update + " last_version = " + last_version
648 + " cur_worker_name = " + cur_worker_name + " status = " + status + " incarnation = "
649 + incarnation + " resubmits = " + unforcedResubmits.get() + " batch = " + batch);
650 }
651
652 public Task() {
653 incarnation = 0;
654 last_version = -1;
655 status = IN_PROGRESS;
656 setUnassigned();
657 }
658
659 public boolean isOrphan() {
660 return (batch == null || batch.isDead);
661 }
662
663 public boolean isUnassigned() {
664 return (cur_worker_name == null);
665 }
666
667 public void heartbeatNoDetails(long time) {
668 last_update = time;
669 }
670
671 public void heartbeat(long time, int version, ServerName worker) {
672 last_version = version;
673 last_update = time;
674 cur_worker_name = worker;
675 }
676
677 public void setUnassigned() {
678 cur_worker_name = null;
679 last_update = -1;
680 }
681 }
682
683
684
685
686 private class TimeoutMonitor extends Chore {
687 private long lastLog = 0;
688
689 public TimeoutMonitor(final int period, Stoppable stopper) {
690 super("SplitLogManager Timeout Monitor", period, stopper);
691 }
692
693 @Override
694 protected void chore() {
695 int resubmitted = 0;
696 int unassigned = 0;
697 int tot = 0;
698 boolean found_assigned_task = false;
699 Set<ServerName> localDeadWorkers;
700
701 synchronized (deadWorkersLock) {
702 localDeadWorkers = deadWorkers;
703 deadWorkers = null;
704 }
705
706 for (Map.Entry<String, Task> e : tasks.entrySet()) {
707 String path = e.getKey();
708 Task task = e.getValue();
709 ServerName cur_worker = task.cur_worker_name;
710 tot++;
711
712
713
714
715
716 if (task.isUnassigned()) {
717 unassigned++;
718 continue;
719 }
720 found_assigned_task = true;
721 if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
722 SplitLogCounters.tot_mgr_resubmit_dead_server_task.incrementAndGet();
723 if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
724 .getSplitLogManagerCoordination().resubmitTask(path, task, FORCE)) {
725 resubmitted++;
726 } else {
727 handleDeadWorker(cur_worker);
728 LOG.warn("Failed to resubmit task " + path + " owned by dead " + cur_worker
729 + ", will retry.");
730 }
731 } else if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
732 .getSplitLogManagerCoordination().resubmitTask(path, task, CHECK)) {
733 resubmitted++;
734 }
735 }
736 if (tot > 0) {
737 long now = EnvironmentEdgeManager.currentTime();
738 if (now > lastLog + 5000) {
739 lastLog = now;
740 LOG.info("total tasks = " + tot + " unassigned = " + unassigned + " tasks=" + tasks);
741 }
742 }
743 if (resubmitted > 0) {
744 LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
745 }
746
747
748
749
750
751
752
753
754
755 if (tot > 0
756 && !found_assigned_task
757 && ((EnvironmentEdgeManager.currentTime() - lastTaskCreateTime) > unassignedTimeout)) {
758 for (Map.Entry<String, Task> e : tasks.entrySet()) {
759 String key = e.getKey();
760 Task task = e.getValue();
761
762
763
764
765 if (task.isUnassigned() && (task.status != FAILURE)) {
766
767 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
768 .getSplitLogManagerCoordination().checkTaskStillAvailable(key);
769 }
770 }
771 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
772 .getSplitLogManagerCoordination().checkTasks();
773 SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
774 LOG.debug("resubmitting unassigned task(s) after timeout");
775 }
776 Set<String> failedDeletions =
777 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
778 .getSplitLogManagerCoordination().getDetails().getFailedDeletions();
779
780 if (failedDeletions.size() > 0) {
781 List<String> tmpPaths = new ArrayList<String>(failedDeletions);
782 for (String tmpPath : tmpPaths) {
783
784 ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
785 .getSplitLogManagerCoordination().deleteTask(tmpPath);
786 }
787 failedDeletions.removeAll(tmpPaths);
788 }
789
790
791 long timeInterval =
792 EnvironmentEdgeManager.currentTime()
793 - ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
794 .getSplitLogManagerCoordination().getLastRecoveryTime();
795 if (!failedRecoveringRegionDeletions.isEmpty()
796 || (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) {
797
798 if (!failedRecoveringRegionDeletions.isEmpty()) {
799 List<Pair<Set<ServerName>, Boolean>> previouslyFailedDeletions =
800 new ArrayList<Pair<Set<ServerName>, Boolean>>(failedRecoveringRegionDeletions);
801 failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletions);
802 for (Pair<Set<ServerName>, Boolean> failedDeletion : previouslyFailedDeletions) {
803 removeRecoveringRegions(failedDeletion.getFirst(), failedDeletion.getSecond());
804 }
805 } else {
806 removeRecoveringRegions(null, null);
807 }
808 }
809 }
810 }
811
812 public enum ResubmitDirective {
813 CHECK(), FORCE();
814 }
815
816 public enum TerminationStatus {
817 IN_PROGRESS("in_progress"), SUCCESS("success"), FAILURE("failure"), DELETED("deleted");
818
819 String statusMsg;
820
821 TerminationStatus(String msg) {
822 statusMsg = msg;
823 }
824
825 @Override
826 public String toString() {
827 return statusMsg;
828 }
829 }
830 }