View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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.Arrays;
31  import java.util.Collections;
32  import java.util.HashSet;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.Set;
36  import java.util.concurrent.ConcurrentHashMap;
37  import java.util.concurrent.ConcurrentMap;
38  import java.util.concurrent.atomic.AtomicInteger;
39  import java.util.concurrent.locks.ReentrantLock;
40  
41  import org.apache.commons.logging.Log;
42  import org.apache.commons.logging.LogFactory;
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.ChoreService;
49  import org.apache.hadoop.hbase.HRegionInfo;
50  import org.apache.hadoop.hbase.ScheduledChore;
51  import org.apache.hadoop.hbase.Server;
52  import org.apache.hadoop.hbase.ServerName;
53  import org.apache.hadoop.hbase.SplitLogCounters;
54  import org.apache.hadoop.hbase.Stoppable;
55  import org.apache.hadoop.hbase.classification.InterfaceAudience;
56  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
57  import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
58  import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
59  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
60  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
61  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
62  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
63  import org.apache.hadoop.hbase.util.FSUtils;
64  import org.apache.hadoop.hbase.util.Pair;
65  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
66  import org.apache.hadoop.hbase.wal.WALFactory;
67  
68  /**
69   * Distributes the task of log splitting to the available region servers.
70   * Coordination happens via coordination engine. For every log file that has to be split a
71   * task is created. SplitLogWorkers race to grab a task.
72   *
73   * <p>SplitLogManager monitors the tasks that it creates using the
74   * timeoutMonitor thread. If a task's progress is slow then
75   * {@link SplitLogManagerCoordination#checkTasks} will take away the
76   * task from the owner {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker} 
77   * and the task will be up for grabs again. When the task is done then it is deleted 
78   * by SplitLogManager.
79   *
80   * <p>Clients call {@link #splitLogDistributed(Path)} to split a region server's
81   * log files. The caller thread waits in this method until all the log files
82   * have been split.
83   *
84   * <p>All the coordination calls made by this class are asynchronous. This is mainly
85   * to help reduce response time seen by the callers.
86   *
87   * <p>There is race in this design between the SplitLogManager and the
88   * SplitLogWorker. SplitLogManager might re-queue a task that has in reality
89   * already been completed by a SplitLogWorker. We rely on the idempotency of
90   * the log splitting task for correctness.
91   *
92   * <p>It is also assumed that every log splitting task is unique and once
93   * completed (either with success or with error) it will be not be submitted
94   * again. If a task is resubmitted then there is a risk that old "delete task"
95   * can delete the re-submission.
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   private final ChoreService choreService;
106 
107   public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); // 3 min
108 
109   private long unassignedTimeout;
110   private long lastTaskCreateTime = Long.MAX_VALUE;
111   private long checkRecoveringTimeThreshold = 15000; // 15 seconds
112   private final List<Pair<Set<ServerName>, Boolean>> failedRecoveringRegionDeletions = Collections
113       .synchronizedList(new ArrayList<Pair<Set<ServerName>, Boolean>>());
114 
115   /**
116    * In distributedLogReplay mode, we need touch both splitlog and recovering-regions znodes in one
117    * operation. So the lock is used to guard such cases.
118    */
119   protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
120 
121   private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
122   private TimeoutMonitor timeoutMonitor;
123 
124   private volatile Set<ServerName> deadWorkers = null;
125   private final Object deadWorkersLock = new Object();
126 
127   /**
128    * Its OK to construct this object even when region-servers are not online. It does lookup the
129    * orphan tasks in coordination engine but it doesn't block waiting for them to be done.
130    * @param server the server instance
131    * @param conf the HBase configuration
132    * @param stopper the stoppable in case anything is wrong
133    * @param master the master services
134    * @param serverName the master server name
135    * @throws IOException
136    */
137   public SplitLogManager(Server server, Configuration conf, Stoppable stopper,
138       MasterServices master, ServerName serverName) throws IOException {
139     this.server = server;
140     this.conf = conf;
141     this.stopper = stopper;
142     this.choreService = new ChoreService(serverName.toString() + "_splitLogManager_");
143     if (server.getCoordinatedStateManager() != null) {
144       SplitLogManagerCoordination coordination =
145           ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
146               .getSplitLogManagerCoordination();
147       Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());
148       SplitLogManagerDetails details =
149           new SplitLogManagerDetails(tasks, master, failedDeletions, serverName);
150       coordination.setDetails(details);
151       coordination.init();
152       // Determine recovery mode
153     }
154     this.unassignedTimeout =
155         conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
156     this.timeoutMonitor =
157         new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000),
158             stopper);
159     choreService.scheduleChore(timeoutMonitor);
160   }
161 
162   private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
163     return getFileList(conf, logDirs, filter);
164   }
165 
166   /**
167    * Get a list of paths that need to be split given a set of server-specific directories and
168    * optionally  a filter.
169    *
170    * See {@link DefaultWALProvider#getServerNameFromWALDirectoryName} for more info on directory
171    * layout.
172    *
173    * Should be package-private, but is needed by
174    * {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
175    *     Configuration, WALFactory)} for tests.
176    */
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    * @param logDir one region sever wal dir path in .logs
200    * @throws IOException if there was an error while splitting any log file
201    * @return cumulative size of the logfiles split
202    * @throws IOException
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    * The caller will block until all the log files of the given region server have been processed -
212    * successfully split or an error is encountered - by an available worker region server. This
213    * method must only be called after the region servers have been brought online.
214    * @param logDirs List of log dirs to split
215    * @throws IOException If there was an error while splitting any log file
216    * @return cumulative size of the logfiles split
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         // ignore invalid format error.
231         LOG.warn("Cannot parse server name from " + logDir);
232       }
233     }
234     return splitLogDistributed(serverNames, logDirs, null);
235   }
236 
237   /**
238    * The caller will block until all the hbase:meta log files of the given region server have been
239    * processed - successfully split or an error is encountered - by an available worker region
240    * server. This method must only be called after the region servers have been brought online.
241    * @param logDirs List of log dirs to split
242    * @param filter the Path filter to select specific files for considering
243    * @throws IOException If there was an error while splitting any log file
244    * @return cumulative size of the logfiles split
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     SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
253     LOG.info("Started splitting " + logfiles.length + " logs in " + logDirs +
254       " for " + serverNames);
255     long t = EnvironmentEdgeManager.currentTime();
256     long totalSize = 0;
257     TaskBatch batch = new TaskBatch();
258     Boolean isMetaRecovery = (filter == null) ? null : false;
259     for (FileStatus lf : logfiles) {
260       // TODO If the log file is still being written to - which is most likely
261       // the case for the last log file - then its length will show up here
262       // as zero. The size of such a file can only be retrieved after
263       // recover-lease is done. totalSize will be under in most cases and the
264       // metrics that it drives will also be under-reported.
265       totalSize += lf.getLen();
266       String pathToLog = FSUtils.removeWALRootPath(lf.getPath(), conf);
267       if (!enqueueSplitTask(pathToLog, batch)) {
268         throw new IOException("duplicate log split scheduled for " + lf.getPath());
269       }
270     }
271     waitForSplittingCompletion(batch, status);
272     // remove recovering regions
273     if (filter == MasterFileSystem.META_FILTER /* reference comparison */) {
274       // we split meta regions and user regions separately therefore logfiles are either all for
275       // meta or user regions but won't for both( we could have mixed situations in tests)
276       isMetaRecovery = true;
277     }
278     removeRecoveringRegions(serverNames, isMetaRecovery);
279 
280     if (batch.done != batch.installed) {
281       batch.isDead = true;
282       SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
283       LOG.warn("error while splitting logs in " + logDirs + " installed = " + batch.installed
284           + " but only " + batch.done + " done");
285       String msg = "error or interrupted while splitting logs in " + logDirs + " Task = " + batch;
286       status.abort(msg);
287       throw new IOException(msg);
288     }
289     for (Path logDir : logDirs) {
290       status.setStatus("Cleaning up log directory...");
291       final FileSystem fs = logDir.getFileSystem(conf);
292       try {
293         if (fs.exists(logDir) && !fs.delete(logDir, false)) {
294           LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
295         }
296       } catch (IOException ioe) {
297         FileStatus[] files = fs.listStatus(logDir);
298         if (files != null && files.length > 0) {
299           LOG.warn("Returning success without actually splitting and "
300               + "deleting all the log files in path " + logDir + ": "
301               + Arrays.toString(files), ioe);
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    * Add a task entry to coordination if it is not already there.
319    * @param taskname the path of the log to be split
320    * @param batch the batch this task belongs to
321    * @return true if a new entry is created, false if it is already there.
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       // publish the task in the coordination engine
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   ConcurrentMap<String, Task> getTasks() {
379     return tasks;
380   }
381 
382   private int activeTasks(final TaskBatch batch) {
383     int count = 0;
384     for (Task t : tasks.values()) {
385       if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
386         count++;
387       }
388     }
389     return count;
390 
391   }
392 
393   /**
394    * It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
395    * region server hosting the region can allow reads to the recovered region
396    * @param serverNames servers which are just recovered
397    * @param isMetaRecovery whether current recovery is for the meta region on
398    *          <code>serverNames<code>
399    */
400   private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
401     if (!isLogReplaying()) {
402       // the function is only used in WALEdit direct replay mode
403       return;
404     }
405     if (serverNames == null || serverNames.isEmpty()) return;
406 
407     Set<String> recoveredServerNameSet = new HashSet<String>();
408     for (ServerName tmpServerName : serverNames) {
409       recoveredServerNameSet.add(tmpServerName.getServerName());
410     }
411    
412     this.recoveringRegionLock.lock();
413     try {
414       ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
415           .getSplitLogManagerCoordination().removeRecoveringRegions(recoveredServerNameSet,
416             isMetaRecovery);
417     } catch (IOException e) {
418       LOG.warn("removeRecoveringRegions got exception. Will retry", e);
419       if (serverNames != null && !serverNames.isEmpty()) {
420         this.failedRecoveringRegionDeletions.add(new Pair<Set<ServerName>, Boolean>(serverNames,
421             isMetaRecovery));
422       }
423     } finally {
424       this.recoveringRegionLock.unlock();
425     }
426   }
427 
428   /**
429    * It removes stale recovering regions under /hbase/recovering-regions/[encoded region name]
430    * during master initialization phase.
431    * @param failedServers A set of known failed servers
432    * @throws IOException
433    */
434   void removeStaleRecoveringRegions(final Set<ServerName> failedServers) throws IOException,
435       InterruptedIOException {
436     Set<String> knownFailedServers = new HashSet<String>();
437     if (failedServers != null) {
438       for (ServerName tmpServerName : failedServers) {
439         knownFailedServers.add(tmpServerName.getServerName());
440       }
441     }
442 
443     this.recoveringRegionLock.lock();
444     try {
445       ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
446           .getSplitLogManagerCoordination().removeStaleRecoveringRegions(knownFailedServers);
447     } finally {
448       this.recoveringRegionLock.unlock();
449     }
450   }
451 
452   /**
453    * @param path
454    * @param batch
455    * @return null on success, existing task on error
456    */
457   private Task createTaskIfAbsent(String path, TaskBatch batch) {
458     Task oldtask;
459     // batch.installed is only changed via this function and
460     // a single thread touches batch.installed.
461     Task newtask = new Task();
462     newtask.batch = batch;
463     oldtask = tasks.putIfAbsent(path, newtask);
464     if (oldtask == null) {
465       batch.installed++;
466       return null;
467     }
468     // new task was not used.
469     synchronized (oldtask) {
470       if (oldtask.isOrphan()) {
471         if (oldtask.status == SUCCESS) {
472           // The task is already done. Do not install the batch for this
473           // task because it might be too late for setDone() to update
474           // batch.done. There is no need for the batch creator to wait for
475           // this task to complete.
476           return (null);
477         }
478         if (oldtask.status == IN_PROGRESS) {
479           oldtask.batch = batch;
480           batch.installed++;
481           LOG.debug("Previously orphan task " + path + " is now being waited upon");
482           return null;
483         }
484         while (oldtask.status == FAILURE) {
485           LOG.debug("wait for status of task " + path + " to change to DELETED");
486           SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
487           try {
488             oldtask.wait();
489           } catch (InterruptedException e) {
490             Thread.currentThread().interrupt();
491             LOG.warn("Interrupted when waiting for znode delete callback");
492             // fall through to return failure
493             break;
494           }
495         }
496         if (oldtask.status != DELETED) {
497           LOG.warn("Failure because previously failed task"
498               + " state still present. Waiting for znode delete callback" + " path=" + path);
499           return oldtask;
500         }
501         // reinsert the newTask and it must succeed this time
502         Task t = tasks.putIfAbsent(path, newtask);
503         if (t == null) {
504           batch.installed++;
505           return null;
506         }
507         LOG.fatal("Logic error. Deleted task still present in tasks map");
508         assert false : "Deleted task still present in tasks map";
509         return t;
510       }
511       LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
512       return oldtask;
513     }
514   }
515 
516   Task findOrCreateOrphanTask(String path) {
517     Task orphanTask = new Task();
518     Task task;
519     task = tasks.putIfAbsent(path, orphanTask);
520     if (task == null) {
521       LOG.info("creating orphan task " + path);
522       SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
523       task = orphanTask;
524     }
525     return task;
526   }
527 
528   public void stop() {
529     if (choreService != null) {
530       choreService.shutdown();
531     }
532     if (timeoutMonitor != null) {
533       timeoutMonitor.cancel(true);
534     }
535   }
536 
537   void handleDeadWorker(ServerName workerName) {
538     // resubmit the tasks on the TimeoutMonitor thread. Makes it easier
539     // to reason about concurrency. Makes it easier to retry.
540     synchronized (deadWorkersLock) {
541       if (deadWorkers == null) {
542         deadWorkers = new HashSet<ServerName>(100);
543       }
544       deadWorkers.add(workerName);
545     }
546     LOG.info("dead splitlog worker " + workerName);
547   }
548 
549   void handleDeadWorkers(Set<ServerName> serverNames) {
550     synchronized (deadWorkersLock) {
551       if (deadWorkers == null) {
552         deadWorkers = new HashSet<ServerName>(100);
553       }
554       deadWorkers.addAll(serverNames);
555     }
556     LOG.info("dead splitlog workers " + serverNames);
557   }
558 
559   /**
560    * This function is to set recovery mode from outstanding split log tasks from before or current
561    * configuration setting
562    * @param isForInitialization
563    * @throws IOException throws if it's impossible to set recovery mode
564    */
565   public void setRecoveryMode(boolean isForInitialization) throws IOException {
566     ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
567         .getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
568 
569   }
570 
571   public void markRegionsRecovering(ServerName server, Set<HRegionInfo> userRegions)
572       throws InterruptedIOException, IOException {
573     if (userRegions == null || (!isLogReplaying())) {
574       return;
575     }
576     try {
577       this.recoveringRegionLock.lock();
578       // mark that we're creating recovering regions
579       ((BaseCoordinatedStateManager) this.server.getCoordinatedStateManager())
580           .getSplitLogManagerCoordination().markRegionsRecovering(server, userRegions);
581     } finally {
582       this.recoveringRegionLock.unlock();
583     }
584 
585   }
586 
587   /**
588    * @return whether log is replaying
589    */
590   public boolean isLogReplaying() {
591     if (server.getCoordinatedStateManager() == null) return false;
592     return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
593         .getSplitLogManagerCoordination().isReplaying();
594   }
595 
596   /**
597    * @return whether log is splitting
598    */
599   public boolean isLogSplitting() {
600     if (server.getCoordinatedStateManager() == null) return false;
601     return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
602         .getSplitLogManagerCoordination().isSplitting();
603   }
604 
605   /**
606    * @return the current log recovery mode
607    */
608   public RecoveryMode getRecoveryMode() {
609     return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
610         .getSplitLogManagerCoordination().getRecoveryMode();
611   }
612 
613   /**
614    * Keeps track of the batch of tasks submitted together by a caller in splitLogDistributed().
615    * Clients threads use this object to wait for all their tasks to be done.
616    * <p>
617    * All access is synchronized.
618    */
619   @InterfaceAudience.Private
620   public static class TaskBatch {
621     public int installed = 0;
622     public int done = 0;
623     public int error = 0;
624     public volatile boolean isDead = false;
625 
626     @Override
627     public String toString() {
628       return ("installed = " + installed + " done = " + done + " error = " + error);
629     }
630   }
631 
632   /**
633    * in memory state of an active task.
634    */
635   @InterfaceAudience.Private
636   public static class Task {
637     public volatile long last_update;
638     public volatile int last_version;
639     public volatile ServerName cur_worker_name;
640     public volatile TaskBatch batch;
641     public volatile TerminationStatus status;
642     public volatile AtomicInteger incarnation = new AtomicInteger(0);
643     public final AtomicInteger unforcedResubmits = new AtomicInteger();
644     public volatile boolean resubmitThresholdReached;
645 
646     @Override
647     public String toString() {
648       return ("last_update = " + last_update + " last_version = " + last_version
649           + " cur_worker_name = " + cur_worker_name + " status = " + status + " incarnation = "
650           + incarnation + " resubmits = " + unforcedResubmits.get() + " batch = " + batch);
651     }
652 
653     public Task() {
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    * Periodically checks all active tasks and resubmits the ones that have timed out
685    */
686   private class TimeoutMonitor extends ScheduledChore {
687     private long lastLog = 0;
688 
689     public TimeoutMonitor(final int period, Stoppable stopper) {
690       super("SplitLogManager Timeout Monitor", stopper, period);
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         // don't easily resubmit a task which hasn't been picked up yet. It
712         // might be a long while before a SplitLogWorker is free to pick up a
713         // task. This is because a SplitLogWorker picks up a task one at a
714         // time. If we want progress when there are no region servers then we
715         // will have to run a SplitLogWorker thread in the Master.
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       // If there are pending tasks and all of them have been unassigned for
747       // some time then put up a RESCAN node to ping the workers.
748       // ZKSplitlog.DEFAULT_UNASSIGNED_TIMEOUT is of the order of minutes
749       // because a. it is very unlikely that every worker had a
750       // transient error when trying to grab the task b. if there are no
751       // workers then all tasks wills stay unassigned indefinitely and the
752       // manager will be indefinitely creating RESCAN nodes. TODO may be the
753       // master should spawn both a manager and a worker thread to guarantee
754       // that there is always one worker in the system
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           // we have to do task.isUnassigned() check again because tasks might
762           // have been asynchronously assigned. There is no locking required
763           // for these checks ... it is OK even if tryGetDataSetWatch() is
764           // called unnecessarily for a taskpath
765           if (task.isUnassigned() && (task.status != FAILURE)) {
766             // We just touch the znode to make sure its still there
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       // Retry previously failed deletes
780       if (failedDeletions.size() > 0) {
781         List<String> tmpPaths = new ArrayList<String>(failedDeletions);
782         for (String tmpPath : tmpPaths) {
783           // deleteNode is an async call
784           ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
785               .getSplitLogManagerCoordination().deleteTask(tmpPath);
786         }
787         failedDeletions.removeAll(tmpPaths);
788       }
789 
790       // Garbage collect left-over
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         // inside the function there have more checks before GC anything
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     final String statusMsg;
820 
821     TerminationStatus(String msg) {
822       statusMsg = msg;
823     }
824 
825     @Override
826     public String toString() {
827       return statusMsg;
828     }
829   }
830 }