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 com.google.common.annotations.VisibleForTesting;
28
29  import java.io.IOException;
30  import java.io.InterruptedIOException;
31  import java.util.ArrayList;
32  import java.util.Arrays;
33  import java.util.Collections;
34  import java.util.HashSet;
35  import java.util.List;
36  import java.util.Map;
37  import java.util.Set;
38  import java.util.concurrent.ConcurrentHashMap;
39  import java.util.concurrent.ConcurrentMap;
40  import java.util.concurrent.atomic.AtomicInteger;
41  import java.util.concurrent.locks.ReentrantLock;
42
43  import org.apache.commons.logging.Log;
44  import org.apache.commons.logging.LogFactory;
45  import org.apache.hadoop.conf.Configuration;
46  import org.apache.hadoop.fs.FileStatus;
47  import org.apache.hadoop.fs.FileSystem;
48  import org.apache.hadoop.fs.Path;
49  import org.apache.hadoop.fs.PathFilter;
50  import org.apache.hadoop.hbase.ChoreService;
51  import org.apache.hadoop.hbase.CoordinatedStateManager;
52  import org.apache.hadoop.hbase.HRegionInfo;
53  import org.apache.hadoop.hbase.ScheduledChore;
54  import org.apache.hadoop.hbase.Server;
55  import org.apache.hadoop.hbase.ServerName;
56  import org.apache.hadoop.hbase.SplitLogCounters;
57  import org.apache.hadoop.hbase.Stoppable;
58  import org.apache.hadoop.hbase.classification.InterfaceAudience;
59  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
60  import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination;
61  import org.apache.hadoop.hbase.coordination.SplitLogManagerCoordination.SplitLogManagerDetails;
62  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
63  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
64  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
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.wal.AbstractFSWALProvider;
69  
70  /**
71   * Distributes the task of log splitting to the available region servers.
72   * Coordination happens via coordination engine. For every log file that has to be split a
73   * task is created. SplitLogWorkers race to grab a task.
74   *
75   * <p>SplitLogManager monitors the tasks that it creates using the
76   * timeoutMonitor thread. If a task's progress is slow then
77   * {@link SplitLogManagerCoordination#checkTasks} will take away the
78   * task from the owner {@link org.apache.hadoop.hbase.regionserver.SplitLogWorker}
79   * and the task will be up for grabs again. When the task is done then it is
80   * deleted by SplitLogManager.
81   *
82   * <p>Clients call {@link #splitLogDistributed(Path)} to split a region server's
83   * log files. The caller thread waits in this method until all the log files
84   * have been split.
85   *
86   * <p>All the coordination calls made by this class are asynchronous. This is mainly
87   * to help reduce response time seen by the callers.
88   *
89   * <p>There is race in this design between the SplitLogManager and the
90   * SplitLogWorker. SplitLogManager might re-queue a task that has in reality
91   * already been completed by a SplitLogWorker. We rely on the idempotency of
92   * the log splitting task for correctness.
93   *
94   * <p>It is also assumed that every log splitting task is unique and once
95   * completed (either with success or with error) it will be not be submitted
96   * again. If a task is resubmitted then there is a risk that old "delete task"
97   * can delete the re-submission.
98   */
99  @InterfaceAudience.Private
100 public class SplitLogManager {
101   private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
102 
103   private Server server;
104 
105   private final Stoppable stopper;
106   private final Configuration conf;
107   private final ChoreService choreService;
108 
109   public static final int DEFAULT_UNASSIGNED_TIMEOUT = (3 * 60 * 1000); // 3 min
110 
111   private long unassignedTimeout;
112   private long lastTaskCreateTime = Long.MAX_VALUE;
113   private long checkRecoveringTimeThreshold = 15000; // 15 seconds
114   private final List<Pair<Set<ServerName>, Boolean>> failedRecoveringRegionDeletions = Collections
115       .synchronizedList(new ArrayList<Pair<Set<ServerName>, Boolean>>());
116 
117   /**
118    * In distributedLogReplay mode, we need touch both splitlog and recovering-regions znodes in one
119    * operation. So the lock is used to guard such cases.
120    */
121   protected final ReentrantLock recoveringRegionLock = new ReentrantLock();
122 
123   private final ConcurrentMap<String, Task> tasks = new ConcurrentHashMap<String, Task>();
124   private TimeoutMonitor timeoutMonitor;
125 
126   private volatile Set<ServerName> deadWorkers = null;
127   private final Object deadWorkersLock = new Object();
128 
129   /**
130    * Its OK to construct this object even when region-servers are not online. It does lookup the
131    * orphan tasks in coordination engine but it doesn't block waiting for them to be done.
132    * @param server the server instance
133    * @param conf the HBase configuration
134    * @param stopper the stoppable in case anything is wrong
135    * @param master the master services
136    * @param serverName the master server name
137    * @throws IOException
138    */
139   public SplitLogManager(Server server, Configuration conf, Stoppable stopper,
140       MasterServices master, ServerName serverName) throws IOException {
141     this.server = server;
142     this.conf = conf;
143     this.stopper = stopper;
144     this.choreService = new ChoreService(serverName.toString() + "_splitLogManager_");
145     if (server.getCoordinatedStateManager() != null) {
146       SplitLogManagerCoordination coordination =
147           ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
148               .getSplitLogManagerCoordination();
149       Set<String> failedDeletions = Collections.synchronizedSet(new HashSet<String>());
150       SplitLogManagerDetails details =
151           new SplitLogManagerDetails(tasks, master, failedDeletions, serverName);
152       coordination.setDetails(details);
153       coordination.init();
154       // Determine recovery mode
155     }
156     this.unassignedTimeout =
157         conf.getInt("hbase.splitlog.manager.unassigned.timeout", DEFAULT_UNASSIGNED_TIMEOUT);
158     this.timeoutMonitor =
159         new TimeoutMonitor(conf.getInt("hbase.splitlog.manager.timeoutmonitor.period", 1000),
160             stopper);
161     choreService.scheduleChore(timeoutMonitor);
162   }
163 
164   private FileStatus[] getFileList(List<Path> logDirs, PathFilter filter) throws IOException {
165     return getFileList(conf, logDirs, filter);
166   }
167 
168   /**
169    * Get a list of paths that need to be split given a set of server-specific directories and
170    * optionally  a filter.
171    *
172    * See {@link AbstractFSWALProvider#getServerNameFromWALDirectoryName} for more info on directory
173    * layout.
174    *
175    * Should be package-private, but is needed by
176    * {@link org.apache.hadoop.hbase.wal.WALSplitter#split(Path, Path, Path, FileSystem,
177    *     Configuration, org.apache.hadoop.hbase.wal.WALFactory)} for tests.
178    */
179   @VisibleForTesting
180   public static FileStatus[] getFileList(final Configuration conf, final List<Path> logDirs,
181       final PathFilter filter)
182       throws IOException {
183     List<FileStatus> fileStatus = new ArrayList<FileStatus>();
184     for (Path logDir : logDirs) {
185       final FileSystem fs = logDir.getFileSystem(conf);
186       if (!fs.exists(logDir)) {
187         LOG.warn(logDir + " doesn't exist. Nothing to do!");
188         continue;
189       }
190       FileStatus[] logfiles = FSUtils.listStatus(fs, logDir, filter);
191       if (logfiles == null || logfiles.length == 0) {
192         LOG.info(logDir + " is empty dir, no logs to split");
193       } else {
194         Collections.addAll(fileStatus, logfiles);
195       }
196     }
197     FileStatus[] a = new FileStatus[fileStatus.size()];
198     return fileStatus.toArray(a);
199   }
200 
201   /**
202    * @param logDir one region sever wal dir path in .logs
203    * @throws IOException if there was an error while splitting any log file
204    * @return cumulative size of the logfiles split
205    * @throws IOException
206    */
207   public long splitLogDistributed(final Path logDir) throws IOException {
208     List<Path> logDirs = new ArrayList<Path>();
209     logDirs.add(logDir);
210     return splitLogDistributed(logDirs);
211   }
212 
213   /**
214    * The caller will block until all the log files of the given region server have been processed -
215    * successfully split or an error is encountered - by an available worker region server. This
216    * method must only be called after the region servers have been brought online.
217    * @param logDirs List of log dirs to split
218    * @throws IOException If there was an error while splitting any log file
219    * @return cumulative size of the logfiles split
220    */
221   public long splitLogDistributed(final List<Path> logDirs) throws IOException {
222     if (logDirs.isEmpty()) {
223       return 0;
224     }
225     Set<ServerName> serverNames = new HashSet<ServerName>();
226     for (Path logDir : logDirs) {
227       try {
228         ServerName serverName = AbstractFSWALProvider.getServerNameFromWALDirectoryName(logDir);
229         if (serverName != null) {
230           serverNames.add(serverName);
231         }
232       } catch (IllegalArgumentException e) {
233         // ignore invalid format error.
234         LOG.warn("Cannot parse server name from " + logDir);
235       }
236     }
237     return splitLogDistributed(serverNames, logDirs, null);
238   }
239 
240   /**
241    * The caller will block until all the hbase:meta log files of the given region server have been
242    * processed - successfully split or an error is encountered - by an available worker region
243    * server. This method must only be called after the region servers have been brought online.
244    * @param logDirs List of log dirs to split
245    * @param filter the Path filter to select specific files for considering
246    * @throws IOException If there was an error while splitting any log file
247    * @return cumulative size of the logfiles split
248    */
249   public long splitLogDistributed(final Set<ServerName> serverNames, final List<Path> logDirs,
250       PathFilter filter) throws IOException {
251     MonitoredTask status = TaskMonitor.get().createStatus("Doing distributed log split in " +
252       logDirs + " for serverName=" + serverNames);
253     FileStatus[] logfiles = getFileList(logDirs, filter);
254     status.setStatus("Checking directory contents...");
255     SplitLogCounters.tot_mgr_log_split_batch_start.incrementAndGet();
256     LOG.info("Started splitting " + logfiles.length + " logs in " + logDirs +
257       " for " + serverNames);
258     long t = EnvironmentEdgeManager.currentTime();
259     long totalSize = 0;
260     TaskBatch batch = new TaskBatch();
261     Boolean isMetaRecovery = (filter == null) ? null : false;
262     for (FileStatus lf : logfiles) {
263       // TODO If the log file is still being written to - which is most likely
264       // the case for the last log file - then its length will show up here
265       // as zero. The size of such a file can only be retrieved after
266       // recover-lease is done. totalSize will be under in most cases and the
267       // metrics that it drives will also be under-reported.
268       totalSize += lf.getLen();
269       String pathToLog = FSUtils.removeRootPath(lf.getPath(), conf);
270       if (!enqueueSplitTask(pathToLog, batch)) {
271         throw new IOException("duplicate log split scheduled for " + lf.getPath());
272       }
273     }
274     waitForSplittingCompletion(batch, status);
275     // remove recovering regions
276     if (filter == MasterWalManager.META_FILTER /* reference comparison */) {
277       // we split meta regions and user regions separately therefore logfiles are either all for
278       // meta or user regions but won't for both( we could have mixed situations in tests)
279       isMetaRecovery = true;
280     }
281     removeRecoveringRegions(serverNames, isMetaRecovery);
282 
283     if (batch.done != batch.installed) {
284       batch.isDead = true;
285       SplitLogCounters.tot_mgr_log_split_batch_err.incrementAndGet();
286       LOG.warn("error while splitting logs in " + logDirs + " installed = " + batch.installed
287           + " but only " + batch.done + " done");
288       String msg = "error or interrupted while splitting logs in " + logDirs + " Task = " + batch;
289       status.abort(msg);
290       throw new IOException(msg);
291     }
292     for (Path logDir : logDirs) {
293       status.setStatus("Cleaning up log directory...");
294       final FileSystem fs = logDir.getFileSystem(conf);
295       try {
296         if (fs.exists(logDir) && !fs.delete(logDir, false)) {
297           LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
298         }
299       } catch (IOException ioe) {
300         FileStatus[] files = fs.listStatus(logDir);
301         if (files != null && files.length > 0) {
302           LOG.warn("Returning success without actually splitting and "
303               + "deleting all the log files in path " + logDir + ": "
304               + Arrays.toString(files), ioe);
305         } else {
306           LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
307         }
308       }
309       SplitLogCounters.tot_mgr_log_split_batch_success.incrementAndGet();
310     }
311     String msg =
312         "finished splitting (more than or equal to) " + totalSize + " bytes in " + batch.installed
313             + " log files in " + logDirs + " in "
314             + (EnvironmentEdgeManager.currentTime() - t) + "ms";
315     status.markComplete(msg);
316     LOG.info(msg);
317     return totalSize;
318   }
319 
320   /**
321    * Add a task entry to coordination if it is not already there.
322    * @param taskname the path of the log to be split
323    * @param batch the batch this task belongs to
324    * @return true if a new entry is created, false if it is already there.
325    */
326   boolean enqueueSplitTask(String taskname, TaskBatch batch) {
327     lastTaskCreateTime = EnvironmentEdgeManager.currentTime();
328     String task =
329         ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
330             .getSplitLogManagerCoordination().prepareTask(taskname);
331     Task oldtask = createTaskIfAbsent(task, batch);
332     if (oldtask == null) {
333       // publish the task in the coordination engine
334       ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
335           .getSplitLogManagerCoordination().submitTask(task);
336       return true;
337     }
338     return false;
339   }
340 
341   private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {
342     synchronized (batch) {
343       while ((batch.done + batch.error) != batch.installed) {
344         try {
345           status.setStatus("Waiting for distributed tasks to finish. " + " scheduled="
346               + batch.installed + " done=" + batch.done + " error=" + batch.error);
347           int remaining = batch.installed - (batch.done + batch.error);
348           int actual = activeTasks(batch);
349           if (remaining != actual) {
350             LOG.warn("Expected " + remaining + " active tasks, but actually there are " + actual);
351           }
352           int remainingTasks =
353               ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
354                   .getSplitLogManagerCoordination().remainingTasksInCoordination();
355           if (remainingTasks >= 0 && actual > remainingTasks) {
356             LOG.warn("Expected at least" + actual + " tasks remaining, but actually there are "
357                 + remainingTasks);
358           }
359           if (remainingTasks == 0 || actual == 0) {
360             LOG.warn("No more task remaining, splitting "
361                 + "should have completed. Remaining tasks is " + remainingTasks
362                 + ", active tasks in map " + actual);
363             if (remainingTasks == 0 && actual == 0) {
364               return;
365             }
366           }
367           batch.wait(100);
368           if (stopper.isStopped()) {
369             LOG.warn("Stopped while waiting for log splits to be completed");
370             return;
371           }
372         } catch (InterruptedException e) {
373           LOG.warn("Interrupted while waiting for log splits to be completed");
374           Thread.currentThread().interrupt();
375           return;
376         }
377       }
378     }
379   }
380 
381   @VisibleForTesting
382   ConcurrentMap<String, Task> getTasks() {
383     return tasks;
384   }
385 
386   private int activeTasks(final TaskBatch batch) {
387     int count = 0;
388     for (Task t : tasks.values()) {
389       if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
390         count++;
391       }
392     }
393     return count;
394 
395   }
396 
397   /**
398    * It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
399    * region server hosting the region can allow reads to the recovered region
400    * @param serverNames servers which are just recovered
401    * @param isMetaRecovery whether current recovery is for the meta region on {@code serverNames}
402    */
403   private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
404     if (!isLogReplaying()) {
405       // the function is only used in WALEdit direct replay mode
406       return;
407     }
408     if (serverNames == null || serverNames.isEmpty()) return;
409 
410     Set<String> recoveredServerNameSet = new HashSet<String>();
411     for (ServerName tmpServerName : serverNames) {
412       recoveredServerNameSet.add(tmpServerName.getServerName());
413     }
414
415     this.recoveringRegionLock.lock();
416     try {
417       ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
418           .getSplitLogManagerCoordination().removeRecoveringRegions(recoveredServerNameSet,
419             isMetaRecovery);
420     } catch (IOException e) {
421       LOG.warn("removeRecoveringRegions got exception. Will retry", e);
422       if (serverNames != null && !serverNames.isEmpty()) {
423         this.failedRecoveringRegionDeletions.add(new Pair<Set<ServerName>, Boolean>(serverNames,
424             isMetaRecovery));
425       }
426     } finally {
427       this.recoveringRegionLock.unlock();
428     }
429   }
430 
431   /**
432    * It removes stale recovering regions under /hbase/recovering-regions/[encoded region name]
433    * during master initialization phase.
434    * @param failedServers A set of known failed servers
435    * @throws IOException
436    */
437   void removeStaleRecoveringRegions(final Set<ServerName> failedServers) throws IOException,
438       InterruptedIOException {
439     Set<String> knownFailedServers = new HashSet<String>();
440     if (failedServers != null) {
441       for (ServerName tmpServerName : failedServers) {
442         knownFailedServers.add(tmpServerName.getServerName());
443       }
444     }
445 
446     this.recoveringRegionLock.lock();
447     try {
448       ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
449           .getSplitLogManagerCoordination().removeStaleRecoveringRegions(knownFailedServers);
450     } finally {
451       this.recoveringRegionLock.unlock();
452     }
453   }
454 
455   /**
456    * @param path
457    * @param batch
458    * @return null on success, existing task on error
459    */
460   private Task createTaskIfAbsent(String path, TaskBatch batch) {
461     Task oldtask;
462     // batch.installed is only changed via this function and
463     // a single thread touches batch.installed.
464     Task newtask = new Task();
465     newtask.batch = batch;
466     oldtask = tasks.putIfAbsent(path, newtask);
467     if (oldtask == null) {
468       batch.installed++;
469       return null;
470     }
471     // new task was not used.
472     synchronized (oldtask) {
473       if (oldtask.isOrphan()) {
474         if (oldtask.status == SUCCESS) {
475           // The task is already done. Do not install the batch for this
476           // task because it might be too late for setDone() to update
477           // batch.done. There is no need for the batch creator to wait for
478           // this task to complete.
479           return (null);
480         }
481         if (oldtask.status == IN_PROGRESS) {
482           oldtask.batch = batch;
483           batch.installed++;
484           LOG.debug("Previously orphan task " + path + " is now being waited upon");
485           return null;
486         }
487         while (oldtask.status == FAILURE) {
488           LOG.debug("wait for status of task " + path + " to change to DELETED");
489           SplitLogCounters.tot_mgr_wait_for_zk_delete.incrementAndGet();
490           try {
491             oldtask.wait();
492           } catch (InterruptedException e) {
493             Thread.currentThread().interrupt();
494             LOG.warn("Interrupted when waiting for znode delete callback");
495             // fall through to return failure
496             break;
497           }
498         }
499         if (oldtask.status != DELETED) {
500           LOG.warn("Failure because previously failed task"
501               + " state still present. Waiting for znode delete callback" + " path=" + path);
502           return oldtask;
503         }
504         // reinsert the newTask and it must succeed this time
505         Task t = tasks.putIfAbsent(path, newtask);
506         if (t == null) {
507           batch.installed++;
508           return null;
509         }
510         LOG.fatal("Logic error. Deleted task still present in tasks map");
511         assert false : "Deleted task still present in tasks map";
512         return t;
513       }
514       LOG.warn("Failure because two threads can't wait for the same task; path=" + path);
515       return oldtask;
516     }
517   }
518 
519   Task findOrCreateOrphanTask(String path) {
520     Task orphanTask = new Task();
521     Task task;
522     task = tasks.putIfAbsent(path, orphanTask);
523     if (task == null) {
524       LOG.info("creating orphan task " + path);
525       SplitLogCounters.tot_mgr_orphan_task_acquired.incrementAndGet();
526       task = orphanTask;
527     }
528     return task;
529   }
530 
531   public void stop() {
532     if (choreService != null) {
533       choreService.shutdown();
534     }
535     if (timeoutMonitor != null) {
536       timeoutMonitor.cancel(true);
537     }
538   }
539 
540   void handleDeadWorker(ServerName workerName) {
541     // resubmit the tasks on the TimeoutMonitor thread. Makes it easier
542     // to reason about concurrency. Makes it easier to retry.
543     synchronized (deadWorkersLock) {
544       if (deadWorkers == null) {
545         deadWorkers = new HashSet<ServerName>(100);
546       }
547       deadWorkers.add(workerName);
548     }
549     LOG.info("dead splitlog worker " + workerName);
550   }
551 
552   void handleDeadWorkers(Set<ServerName> serverNames) {
553     synchronized (deadWorkersLock) {
554       if (deadWorkers == null) {
555         deadWorkers = new HashSet<ServerName>(100);
556       }
557       deadWorkers.addAll(serverNames);
558     }
559     LOG.info("dead splitlog workers " + serverNames);
560   }
561 
562   /**
563    * This function is to set recovery mode from outstanding split log tasks from before or current
564    * configuration setting
565    * @param isForInitialization
566    * @throws IOException throws if it's impossible to set recovery mode
567    */
568   public void setRecoveryMode(boolean isForInitialization) throws IOException {
569     ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
570         .getSplitLogManagerCoordination().setRecoveryMode(isForInitialization);
571 
572   }
573 
574   public void markRegionsRecovering(ServerName server, Set<HRegionInfo> userRegions)
575       throws InterruptedIOException, IOException {
576     if (userRegions == null || (!isLogReplaying())) {
577       return;
578     }
579     try {
580       this.recoveringRegionLock.lock();
581       // mark that we're creating recovering regions
582       ((BaseCoordinatedStateManager) this.server.getCoordinatedStateManager())
583           .getSplitLogManagerCoordination().markRegionsRecovering(server, userRegions);
584     } finally {
585       this.recoveringRegionLock.unlock();
586     }
587 
588   }
589 
590   /**
591    * @return whether log is replaying
592    */
593   public boolean isLogReplaying() {
594     CoordinatedStateManager m = server.getCoordinatedStateManager();
595     if (m == null) return false;
596     return ((BaseCoordinatedStateManager)m).getSplitLogManagerCoordination().isReplaying();
597   }
598 
599   /**
600    * @return whether log is splitting
601    */
602   public boolean isLogSplitting() {
603     if (server.getCoordinatedStateManager() == null) return false;
604     return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
605         .getSplitLogManagerCoordination().isSplitting();
606   }
607 
608   /**
609    * @return the current log recovery mode
610    */
611   public RecoveryMode getRecoveryMode() {
612     return ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
613         .getSplitLogManagerCoordination().getRecoveryMode();
614   }
615 
616   /**
617    * Keeps track of the batch of tasks submitted together by a caller in splitLogDistributed().
618    * Clients threads use this object to wait for all their tasks to be done.
619    * <p>
620    * All access is synchronized.
621    */
622   @InterfaceAudience.Private
623   public static class TaskBatch {
624     public int installed = 0;
625     public int done = 0;
626     public int error = 0;
627     public volatile boolean isDead = false;
628 
629     @Override
630     public String toString() {
631       return ("installed = " + installed + " done = " + done + " error = " + error);
632     }
633   }
634 
635   /**
636    * in memory state of an active task.
637    */
638   @InterfaceAudience.Private
639   public static class Task {
640     public volatile long last_update;
641     public volatile int last_version;
642     public volatile ServerName cur_worker_name;
643     public volatile TaskBatch batch;
644     public volatile TerminationStatus status;
645     public volatile AtomicInteger incarnation = new AtomicInteger(0);
646     public final AtomicInteger unforcedResubmits = new AtomicInteger();
647     public volatile boolean resubmitThresholdReached;
648 
649     @Override
650     public String toString() {
651       return ("last_update = " + last_update + " last_version = " + last_version
652           + " cur_worker_name = " + cur_worker_name + " status = " + status + " incarnation = "
653           + incarnation + " resubmits = " + unforcedResubmits.get() + " batch = " + batch);
654     }
655 
656     public Task() {
657       last_version = -1;
658       status = IN_PROGRESS;
659       setUnassigned();
660     }
661 
662     public boolean isOrphan() {
663       return (batch == null || batch.isDead);
664     }
665 
666     public boolean isUnassigned() {
667       return (cur_worker_name == null);
668     }
669 
670     public void heartbeatNoDetails(long time) {
671       last_update = time;
672     }
673 
674     public void heartbeat(long time, int version, ServerName worker) {
675       last_version = version;
676       last_update = time;
677       cur_worker_name = worker;
678     }
679 
680     public void setUnassigned() {
681       cur_worker_name = null;
682       last_update = -1;
683     }
684   }
685 
686   /**
687    * Periodically checks all active tasks and resubmits the ones that have timed out
688    */
689   private class TimeoutMonitor extends ScheduledChore {
690     private long lastLog = 0;
691 
692     public TimeoutMonitor(final int period, Stoppable stopper) {
693       super("SplitLogManager Timeout Monitor", stopper, period);
694     }
695 
696     @Override
697     protected void chore() {
698       int resubmitted = 0;
699       int unassigned = 0;
700       int tot = 0;
701       boolean found_assigned_task = false;
702       Set<ServerName> localDeadWorkers;
703 
704       synchronized (deadWorkersLock) {
705         localDeadWorkers = deadWorkers;
706         deadWorkers = null;
707       }
708 
709       for (Map.Entry<String, Task> e : tasks.entrySet()) {
710         String path = e.getKey();
711         Task task = e.getValue();
712         ServerName cur_worker = task.cur_worker_name;
713         tot++;
714         // don't easily resubmit a task which hasn't been picked up yet. It
715         // might be a long while before a SplitLogWorker is free to pick up a
716         // task. This is because a SplitLogWorker picks up a task one at a
717         // time. If we want progress when there are no region servers then we
718         // will have to run a SplitLogWorker thread in the Master.
719         if (task.isUnassigned()) {
720           unassigned++;
721           continue;
722         }
723         found_assigned_task = true;
724         if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
725           SplitLogCounters.tot_mgr_resubmit_dead_server_task.incrementAndGet();
726           if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
727               .getSplitLogManagerCoordination().resubmitTask(path, task, FORCE)) {
728             resubmitted++;
729           } else {
730             handleDeadWorker(cur_worker);
731             LOG.warn("Failed to resubmit task " + path + " owned by dead " + cur_worker
732                 + ", will retry.");
733           }
734         } else if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
735             .getSplitLogManagerCoordination().resubmitTask(path, task, CHECK)) {
736           resubmitted++;
737         }
738       }
739       if (tot > 0) {
740         long now = EnvironmentEdgeManager.currentTime();
741         if (now > lastLog + 5000) {
742           lastLog = now;
743           LOG.info("total tasks = " + tot + " unassigned = " + unassigned + " tasks=" + tasks);
744         }
745       }
746       if (resubmitted > 0) {
747         LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
748       }
749       // If there are pending tasks and all of them have been unassigned for
750       // some time then put up a RESCAN node to ping the workers.
751       // ZKSplitlog.DEFAULT_UNASSIGNED_TIMEOUT is of the order of minutes
752       // because a. it is very unlikely that every worker had a
753       // transient error when trying to grab the task b. if there are no
754       // workers then all tasks wills stay unassigned indefinitely and the
755       // manager will be indefinitely creating RESCAN nodes. TODO may be the
756       // master should spawn both a manager and a worker thread to guarantee
757       // that there is always one worker in the system
758       if (tot > 0
759           && !found_assigned_task
760           && ((EnvironmentEdgeManager.currentTime() - lastTaskCreateTime) > unassignedTimeout)) {
761         for (Map.Entry<String, Task> e : tasks.entrySet()) {
762           String key = e.getKey();
763           Task task = e.getValue();
764           // we have to do task.isUnassigned() check again because tasks might
765           // have been asynchronously assigned. There is no locking required
766           // for these checks ... it is OK even if tryGetDataSetWatch() is
767           // called unnecessarily for a taskpath
768           if (task.isUnassigned() && (task.status != FAILURE)) {
769             // We just touch the znode to make sure its still there
770             ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
771                 .getSplitLogManagerCoordination().checkTaskStillAvailable(key);
772           }
773         }
774         ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
775             .getSplitLogManagerCoordination().checkTasks();
776         SplitLogCounters.tot_mgr_resubmit_unassigned.incrementAndGet();
777         LOG.debug("resubmitting unassigned task(s) after timeout");
778       }
779       Set<String> failedDeletions =
780           ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
781               .getSplitLogManagerCoordination().getDetails().getFailedDeletions();
782       // Retry previously failed deletes
783       if (failedDeletions.size() > 0) {
784         List<String> tmpPaths = new ArrayList<String>(failedDeletions);
785         for (String tmpPath : tmpPaths) {
786           // deleteNode is an async call
787           ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
788               .getSplitLogManagerCoordination().deleteTask(tmpPath);
789         }
790         failedDeletions.removeAll(tmpPaths);
791       }
792 
793       // Garbage collect left-over
794       long timeInterval =
795           EnvironmentEdgeManager.currentTime()
796               - ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
797                   .getSplitLogManagerCoordination().getLastRecoveryTime();
798       if (!failedRecoveringRegionDeletions.isEmpty()
799           || (tot == 0 && tasks.size() == 0 && (timeInterval > checkRecoveringTimeThreshold))) {
800         // inside the function there have more checks before GC anything
801         if (!failedRecoveringRegionDeletions.isEmpty()) {
802           List<Pair<Set<ServerName>, Boolean>> previouslyFailedDeletions =
803               new ArrayList<Pair<Set<ServerName>, Boolean>>(failedRecoveringRegionDeletions);
804           failedRecoveringRegionDeletions.removeAll(previouslyFailedDeletions);
805           for (Pair<Set<ServerName>, Boolean> failedDeletion : previouslyFailedDeletions) {
806             removeRecoveringRegions(failedDeletion.getFirst(), failedDeletion.getSecond());
807           }
808         } else {
809           removeRecoveringRegions(null, null);
810         }
811       }
812     }
813   }
814 
815   public enum ResubmitDirective {
816     CHECK(), FORCE();
817   }
818 
819   public enum TerminationStatus {
820     IN_PROGRESS("in_progress"), SUCCESS("success"), FAILURE("failure"), DELETED("deleted");
821 
822     String statusMsg;
823 
824     TerminationStatus(String msg) {
825       statusMsg = msg;
826     }
827 
828     @Override
829     public String toString() {
830       return statusMsg;
831     }
832   }
833 }