View Javadoc

1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
23  
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.Collections;
27  import java.util.HashSet;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.Set;
31  import java.util.concurrent.ConcurrentHashMap;
32  import java.util.concurrent.ConcurrentMap;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.Chore;
41  import org.apache.hadoop.hbase.ServerName;
42  import org.apache.hadoop.hbase.Stoppable;
43  import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
44  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
45  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
46  import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
47  import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
48  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
49  import org.apache.hadoop.hbase.util.FSUtils;
50  import org.apache.hadoop.hbase.util.Threads;
51  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
52  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
53  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
54  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
55  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
56  import org.apache.hadoop.util.StringUtils;
57  import org.apache.zookeeper.AsyncCallback;
58  import org.apache.zookeeper.CreateMode;
59  import org.apache.zookeeper.KeeperException;
60  import org.apache.zookeeper.KeeperException.NoNodeException;
61  import org.apache.zookeeper.ZooDefs.Ids;
62  import org.apache.zookeeper.data.Stat;
63  
64  import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.*;
65  import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*;
66  
67  /**
68   * Distributes the task of log splitting to the available region servers.
69   * Coordination happens via zookeeper. For every log file that has to be split a
70   * znode is created under /hbase/splitlog. SplitLogWorkers race to grab a task.
71   *
72   * SplitLogManager monitors the task znodes that it creates using the
73   * timeoutMonitor thread. If a task's progress is slow then
74   * resubmit(String, boolean) will take away the task from the owner
75   * {@link SplitLogWorker} and the task will be
76   * upforgrabs again. When the task is done then the task's znode is deleted by
77   * SplitLogManager.
78   *
79   * Clients call {@link #splitLogDistributed(Path)} to split a region server's
80   * log files. The caller thread waits in this method until all the log files
81   * have been split.
82   *
83   * All the zookeeper calls made by this class are asynchronous. This is mainly
84   * to help reduce response time seen by the callers.
85   *
86   * There is race in this design between the SplitLogManager and the
87   * SplitLogWorker. SplitLogManager might re-queue a task that has in reality
88   * already been completed by a SplitLogWorker. We rely on the idempotency of
89   * the log splitting task for correctness.
90   *
91   * It is also assumed that every log splitting task is unique and once
92   * completed (either with success or with error) it will be not be submitted
93   * again. If a task is resubmitted then there is a risk that old "delete task"
94   * can delete the re-submission.
95   */
96  public class SplitLogManager extends ZooKeeperListener {
97    private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
98  
99    private final Stoppable stopper;
100   private final String serverName;
101   private final TaskFinisher taskFinisher;
102   private FileSystem fs;
103   private Configuration conf;
104 
105   private long zkretries;
106   private long resubmit_threshold;
107   private long timeout;
108   private long unassignedTimeout;
109   private long lastNodeCreateTime = Long.MAX_VALUE;
110   public boolean ignoreZKDeleteForTesting = false;
111 
112   private ConcurrentMap<String, Task> tasks =
113     new ConcurrentHashMap<String, Task>();
114   private TimeoutMonitor timeoutMonitor;
115 
116   private Set<String> deadWorkers = null;
117   private Object deadWorkersLock = new Object();
118 
119   private Set<String> failedDeletions = null;
120 
121   /**
122    * Wrapper around {@link #SplitLogManager(ZooKeeperWatcher, Configuration,
123    * Stoppable, String, TaskFinisher)} that provides a task finisher for
124    * copying recovered edits to their final destination. The task finisher
125    * has to be robust because it can be arbitrarily restarted or called
126    * multiple times.
127    * 
128    * @param zkw
129    * @param conf
130    * @param stopper
131    * @param serverName
132    */
133   public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
134       Stoppable stopper, String serverName) {
135     this(zkw, conf, stopper, serverName, new TaskFinisher() {
136       @Override
137       public Status finish(String workerName, String logfile) {
138         try {
139           HLogSplitter.finishSplitLogFile(logfile, conf);
140         } catch (IOException e) {
141           LOG.warn("Could not finish splitting of log file " + logfile, e);
142           return Status.ERR;
143         }
144         return Status.DONE;
145       }
146     });
147   }
148 
149   /**
150    * Its OK to construct this object even when region-servers are not online. It
151    * does lookup the orphan tasks in zk but it doesn't block waiting for them
152    * to be done.
153    *
154    * @param zkw
155    * @param conf
156    * @param stopper
157    * @param serverName
158    * @param tf task finisher 
159    */
160   public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
161       Stoppable stopper, String serverName, TaskFinisher tf) {
162     super(zkw);
163     this.taskFinisher = tf;
164     this.conf = conf;
165     this.stopper = stopper;
166     this.zkretries = conf.getLong("hbase.splitlog.zk.retries",
167         ZKSplitLog.DEFAULT_ZK_RETRIES);
168     this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit",
169         ZKSplitLog.DEFAULT_MAX_RESUBMIT);
170     this.timeout = conf.getInt("hbase.splitlog.manager.timeout",
171         ZKSplitLog.DEFAULT_TIMEOUT);
172     this.unassignedTimeout =
173       conf.getInt("hbase.splitlog.manager.unassigned.timeout",
174         ZKSplitLog.DEFAULT_UNASSIGNED_TIMEOUT);
175     LOG.debug("timeout = " + timeout);
176     LOG.debug("unassigned timeout = " + unassignedTimeout);
177 
178     this.serverName = serverName;
179     this.timeoutMonitor = new TimeoutMonitor(
180         conf.getInt("hbase.splitlog.manager.timeoutmonitor.period",
181             1000),
182         stopper);
183 
184     this.failedDeletions = Collections.synchronizedSet(new HashSet<String>());
185   }
186 
187   public void finishInitialization(boolean masterRecovery) {
188     if (!masterRecovery) {
189       Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName
190           + ".splitLogManagerTimeoutMonitor");
191     }
192     // Watcher can be null during tests with Mock'd servers.
193     if (this.watcher != null) {
194       this.watcher.registerListener(this);
195       lookForOrphans();
196     }
197   }
198 
199   private FileStatus[] getFileList(List<Path> logDirs) throws IOException {
200     List<FileStatus> fileStatus = new ArrayList<FileStatus>();
201     for (Path hLogDir : logDirs) {
202       this.fs = hLogDir.getFileSystem(conf);
203       if (!fs.exists(hLogDir)) {
204         LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
205         continue;
206       }
207       // TODO filter filenames?
208       FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, null);
209       if (logfiles == null || logfiles.length == 0) {
210         LOG.info(hLogDir + " is empty dir, no logs to split");
211       } else {
212         for (FileStatus status : logfiles)
213           fileStatus.add(status);
214       }
215     }
216     FileStatus[] a = new FileStatus[fileStatus.size()];
217     return fileStatus.toArray(a);
218   }
219 
220   /**
221    * @param logDir
222    *            one region sever hlog dir path in .logs
223    * @throws IOException
224    *             if there was an error while splitting any log file
225    * @return cumulative size of the logfiles split
226    * @throws IOException 
227    */
228   public long splitLogDistributed(final Path logDir) throws IOException {
229     List<Path> logDirs = new ArrayList<Path>();
230     logDirs.add(logDir);
231     return splitLogDistributed(logDirs);
232   }
233   /**
234    * The caller will block until all the log files of the given region server
235    * have been processed - successfully split or an error is encountered - by an
236    * available worker region server. This method must only be called after the
237    * region servers have been brought online.
238    *
239    * @param logDirs
240    * @throws IOException
241    *          if there was an error while splitting any log file
242    * @return cumulative size of the logfiles split
243    */
244   public long splitLogDistributed(final List<Path> logDirs) throws IOException {
245     MonitoredTask status = TaskMonitor.get().createStatus(
246           "Doing distributed log split in " + logDirs);
247     FileStatus[] logfiles = getFileList(logDirs);
248     status.setStatus("Checking directory contents...");
249     LOG.debug("Scheduling batch of logs to split");
250     tot_mgr_log_split_batch_start.incrementAndGet();
251     LOG.info("started splitting logs in " + logDirs);
252     long t = EnvironmentEdgeManager.currentTimeMillis();
253     long totalSize = 0;
254     TaskBatch batch = new TaskBatch();
255     for (FileStatus lf : logfiles) {
256       // TODO If the log file is still being written to - which is most likely
257       // the case for the last log file - then its length will show up here
258       // as zero. The size of such a file can only be retrieved after
259       // recover-lease is done. totalSize will be under in most cases and the
260       // metrics that it drives will also be under-reported.
261       totalSize += lf.getLen();
262       if (enqueueSplitTask(lf.getPath().toString(), batch) == false) {
263         throw new IOException("duplicate log split scheduled for "
264             + lf.getPath());
265       }
266     }
267     waitForSplittingCompletion(batch, status);
268     if (batch.done != batch.installed) {
269       batch.isDead = true;
270       tot_mgr_log_split_batch_err.incrementAndGet();
271       LOG.warn("error while splitting logs in " + logDirs +
272       " installed = " + batch.installed + " but only " + batch.done + " done");
273       String msg = "error or interrupted while splitting logs in "
274         + logDirs + " Task = " + batch;
275       status.abort(msg);
276       throw new IOException(msg);
277     }
278     for(Path logDir: logDirs){
279       status.setStatus("Cleaning up log directory...");
280       try {
281         if (fs.exists(logDir) && !fs.delete(logDir, false)) {
282           LOG.warn("Unable to delete log src dir. Ignoring. " + logDir);
283         }
284       } catch (IOException ioe) {
285         FileStatus[] files = fs.listStatus(logDir);
286         if (files != null && files.length > 0) {
287           LOG.warn("returning success without actually splitting and " + 
288               "deleting all the log files in path " + logDir);
289         } else {
290           LOG.warn("Unable to delete log src dir. Ignoring. " + logDir, ioe);
291         }
292       }
293       tot_mgr_log_split_batch_success.incrementAndGet();
294     }
295     String msg = "finished splitting (more than or equal to) " + totalSize +
296         " bytes in " + batch.installed + " log files in " + logDirs + " in " +
297         (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms";
298     status.markComplete(msg);
299     LOG.info(msg);
300     return totalSize;
301   }
302 
303   /**
304    * Add a task entry to splitlog znode if it is not already there.
305    * 
306    * @param taskname the path of the log to be split
307    * @param batch the batch this task belongs to
308    * @return true if a new entry is created, false if it is already there.
309    */
310   boolean enqueueSplitTask(String taskname, TaskBatch batch) {
311     tot_mgr_log_split_start.incrementAndGet();
312     String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
313     Task oldtask = createTaskIfAbsent(path, batch);
314     if (oldtask == null) {
315       // publish the task in zk
316       createNode(path, zkretries);
317       return true;
318     }
319     return false;
320   }
321 
322   private void waitForSplittingCompletion(TaskBatch batch, MonitoredTask status) {
323     synchronized (batch) {
324       while ((batch.done + batch.error) != batch.installed) {
325         try {
326           status.setStatus("Waiting for distributed tasks to finish. "
327               + " scheduled=" + batch.installed
328               + " done=" + batch.done
329               + " error=" + batch.error);
330           int remaining = batch.installed - (batch.done + batch.error);
331           int actual = activeTasks(batch);
332           if (remaining != actual) {
333             LOG.warn("Expected " + remaining
334               + " active tasks, but actually there are " + actual);
335           }
336           int remainingInZK = remainingTasksInZK();
337           if (remainingInZK >= 0 && actual > remainingInZK) {
338             LOG.warn("Expected at least" + actual
339               + " tasks in ZK, but actually there are " + remainingInZK);
340           }
341           if (remainingInZK == 0 || actual == 0) {
342             LOG.warn("No more task remaining (ZK or task map), splitting "
343               + "should have completed. Remaining tasks in ZK " + remainingInZK
344               + ", active tasks in map " + actual);
345             if (remainingInZK == 0 && actual == 0) {
346               return;
347             }
348           }
349           batch.wait(100);
350           if (stopper.isStopped()) {
351             LOG.warn("Stopped while waiting for log splits to be completed");
352             return;
353           }
354         } catch (InterruptedException e) {
355           LOG.warn("Interrupted while waiting for log splits to be completed");
356           Thread.currentThread().interrupt();
357           return;
358         }
359       }
360     }
361   }
362 
363   private int activeTasks(final TaskBatch batch) {
364     int count = 0;
365     for (Task t: tasks.values()) {
366       if (t.batch == batch && t.status == TerminationStatus.IN_PROGRESS) {
367         count++;
368       }
369     }
370     return count;
371   }
372 
373   private int remainingTasksInZK() {
374     int count = 0;
375     try {
376       List<String> tasks =
377         ZKUtil.listChildrenNoWatch(watcher, watcher.splitLogZNode);
378       if (tasks != null) {
379         for (String t: tasks) {
380           if (!ZKSplitLog.isRescanNode(watcher, t)) {
381             count++;
382           }
383         }
384       }
385     } catch (KeeperException ke) {
386       LOG.warn("Failed to check remaining tasks", ke);
387       count = -1;
388     }
389     return count;
390   }
391 
392   private void setDone(String path, TerminationStatus status) {
393     Task task = tasks.get(path);
394     if (task == null) {
395       if (!ZKSplitLog.isRescanNode(watcher, path)) {
396         tot_mgr_unacquired_orphan_done.incrementAndGet();
397         LOG.debug("unacquired orphan task is done " + path);
398       }
399     } else {
400       synchronized (task) {
401         if (task.status == IN_PROGRESS) {
402           if (status == SUCCESS) {
403             tot_mgr_log_split_success.incrementAndGet();
404             LOG.info("Done splitting " + path);
405           } else {
406             tot_mgr_log_split_err.incrementAndGet();
407             LOG.warn("Error splitting " + path);
408           }
409           task.status = status;
410           if (task.batch != null) {
411             synchronized (task.batch) {
412               if (status == SUCCESS) {
413                 task.batch.done++;
414               } else {
415                 task.batch.error++;
416               }
417               task.batch.notify();
418             }
419           }
420         }
421       }
422     }
423     // delete the task node in zk. It's an async
424     // call and no one is blocked waiting for this node to be deleted. All
425     // task names are unique (log.<timestamp>) there is no risk of deleting
426     // a future task.
427     // if a deletion fails, TimeoutMonitor will retry the same deletion later
428     deleteNode(path, zkretries);
429     return;
430   }
431 
432   private void createNode(String path, Long retry_count) {
433     ZKUtil.asyncCreate(this.watcher, path,
434         TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(),
435         retry_count);
436     tot_mgr_node_create_queued.incrementAndGet();
437     return;
438   }
439 
440   private void createNodeSuccess(String path) {
441     lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
442     LOG.debug("put up splitlog task at znode " + path);
443     getDataSetWatch(path, zkretries);
444   }
445 
446   private void createNodeFailure(String path) {
447     // TODO the Manager should split the log locally instead of giving up
448     LOG.warn("failed to create task node" + path);
449     setDone(path, FAILURE);
450   }
451 
452 
453   private void getDataSetWatch(String path, Long retry_count) {
454     this.watcher.getRecoverableZooKeeper().getZooKeeper().
455         getData(path, this.watcher,
456         new GetDataAsyncCallback(), retry_count);
457     tot_mgr_get_data_queued.incrementAndGet();
458   }
459 
460   private void tryGetDataSetWatch(String path) {
461     // A negative retry count will lead to ignoring all error processing.
462     this.watcher.getRecoverableZooKeeper().getZooKeeper().
463         getData(path, this.watcher,
464         new GetDataAsyncCallback(), new Long(-1) /* retry count */);
465     tot_mgr_get_data_queued.incrementAndGet();
466   }
467 
468   private void getDataSetWatchSuccess(String path, byte[] data, int version) {
469     if (data == null) {
470       if (version == Integer.MIN_VALUE) {
471         // assume all done. The task znode suddenly disappeared.
472         setDone(path, SUCCESS);
473         return;
474       }
475       tot_mgr_null_data.incrementAndGet();
476       LOG.fatal("logic error - got null data " + path);
477       setDone(path, FAILURE);
478       return;
479     }
480     data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
481     // LOG.debug("set watch on " + path + " got data " + new String(data));
482     if (TaskState.TASK_UNASSIGNED.equals(data)) {
483       LOG.debug("task not yet acquired " + path + " ver = " + version);
484       handleUnassignedTask(path);
485     } else if (TaskState.TASK_OWNED.equals(data)) {
486       heartbeat(path, version,
487           TaskState.TASK_OWNED.getWriterName(data));
488     } else if (TaskState.TASK_RESIGNED.equals(data)) {
489       LOG.info("task " + path + " entered state " + new String(data));
490       resubmitOrFail(path, FORCE);
491     } else if (TaskState.TASK_DONE.equals(data)) {
492       LOG.info("task " + path + " entered state " + new String(data));
493       if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
494         if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data),
495             ZKSplitLog.getFileName(path)) == Status.DONE) {
496           setDone(path, SUCCESS);
497         } else {
498           resubmitOrFail(path, CHECK);
499         }
500       } else {
501         setDone(path, SUCCESS);
502       }
503     } else if (TaskState.TASK_ERR.equals(data)) {
504       LOG.info("task " + path + " entered state " + new String(data));
505       resubmitOrFail(path, CHECK);
506     } else {
507       LOG.fatal("logic error - unexpected zk state for path = " + path
508           + " data = " + new String(data));
509       setDone(path, FAILURE);
510     }
511   }
512 
513   private void getDataSetWatchFailure(String path) {
514     LOG.warn("failed to set data watch " + path);
515     setDone(path, FAILURE);
516   }
517 
518   /**
519    * It is possible for a task to stay in UNASSIGNED state indefinitely - say
520    * SplitLogManager wants to resubmit a task. It forces the task to UNASSIGNED
521    * state but it dies before it could create the RESCAN task node to signal
522    * the SplitLogWorkers to pick up the task. To prevent this scenario the
523    * SplitLogManager resubmits all orphan and UNASSIGNED tasks at startup.
524    *
525    * @param path
526    */
527   private void handleUnassignedTask(String path) {
528     if (ZKSplitLog.isRescanNode(watcher, path)) {
529       return;
530     }
531     Task task = findOrCreateOrphanTask(path);
532     if (task.isOrphan() && (task.incarnation == 0)) {
533       LOG.info("resubmitting unassigned orphan task " + path);
534       // ignore failure to resubmit. The timeout-monitor will handle it later
535       // albeit in a more crude fashion
536       resubmit(path, task, FORCE);
537     }
538   }
539 
540   /**
541    * Helper function to check whether to abandon retries in ZooKeeper AsyncCallback functions
542    * @param statusCode integer value of a ZooKeeper exception code
543    * @param action description message about the retried action
544    * @return true when need to abandon retries, otherwise false
545    */
546   private boolean shouldAbandonRetries(int statusCode, String action) {
547     if (statusCode == KeeperException.Code.SESSIONEXPIRED.intValue()) {
548       LOG.error("ZK session expired. Master is expected to shut down. Abandoning retries for "
549           + "action=" + action);
550       return true;
551     }
552     return false;
553   }
554 
555   private void heartbeat(String path, int new_version,
556       String workerName) {
557     Task task = findOrCreateOrphanTask(path);
558     if (new_version != task.last_version) {
559       if (task.isUnassigned()) {
560         LOG.info("task " + path + " acquired by " + workerName);
561       }
562       task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(),
563           new_version, workerName);
564       tot_mgr_heartbeat.incrementAndGet();
565     } else {
566       // duplicate heartbeats - heartbeats w/o zk node version
567       // changing - are possible. The timeout thread does
568       // getDataSetWatch() just to check whether a node still
569       // exists or not
570     }
571     return;
572   }
573 
574   private boolean resubmit(String path, Task task,
575       ResubmitDirective directive) {
576     // its ok if this thread misses the update to task.deleted. It will
577     // fail later
578     if (task.status != IN_PROGRESS) {
579       return false;
580     }
581     int version;
582     if (directive != FORCE) {
583       if ((EnvironmentEdgeManager.currentTimeMillis() - task.last_update) <
584           timeout) {
585         return false;
586       }
587       if (task.unforcedResubmits >= resubmit_threshold) {
588         if (!task.resubmitThresholdReached) {
589           task.resubmitThresholdReached = true;
590           tot_mgr_resubmit_threshold_reached.incrementAndGet();
591           LOG.info("Skipping resubmissions of task " + path +
592               " because threshold " + resubmit_threshold + " reached");
593         }
594         return false;
595       }
596       // race with heartbeat() that might be changing last_version
597       version = task.last_version;
598     } else {
599       version = -1;
600     }
601     LOG.info("resubmitting task " + path);
602     task.incarnation++;
603     try {
604       // blocking zk call but this is done from the timeout thread
605       if (ZKUtil.setData(this.watcher, path,
606           TaskState.TASK_UNASSIGNED.get(serverName),
607           version) == false) {
608         LOG.debug("failed to resubmit task " + path +
609             " version changed");
610         task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
611         return false;
612       }
613     } catch (NoNodeException e) {
614       LOG.warn("failed to resubmit because znode doesn't exist " + path +
615           " task done (or forced done by removing the znode)");
616       getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
617       return false;
618     } catch (KeeperException.BadVersionException e) {
619       LOG.debug("failed to resubmit task " + path +
620           " version changed");
621       task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
622       return false;
623     } catch (KeeperException e) {
624       tot_mgr_resubmit_failed.incrementAndGet();
625       LOG.warn("failed to resubmit " + path, e);
626       return false;
627     }
628     // don't count forced resubmits
629     if (directive != FORCE) {
630       task.unforcedResubmits++;
631     }
632     task.setUnassigned();
633     createRescanNode(Long.MAX_VALUE);
634     tot_mgr_resubmit.incrementAndGet();
635     return true;
636   }
637 
638   private void resubmitOrFail(String path, ResubmitDirective directive) {
639     if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
640       setDone(path, FAILURE);
641     }
642   }
643 
644   private void deleteNode(String path, Long retries) {
645     tot_mgr_node_delete_queued.incrementAndGet();
646     // Once a task znode is ready for delete, that is it is in the TASK_DONE
647     // state, then no one should be writing to it anymore. That is no one
648     // will be updating the znode version any more.
649     this.watcher.getRecoverableZooKeeper().getZooKeeper().
650       delete(path, -1, new DeleteAsyncCallback(),
651         retries);
652   }
653 
654   private void deleteNodeSuccess(String path) {
655     if (ignoreZKDeleteForTesting) {
656       return;
657     }
658     Task task;
659     task = tasks.remove(path);
660     if (task == null) {
661       if (ZKSplitLog.isRescanNode(watcher, path)) {
662         tot_mgr_rescan_deleted.incrementAndGet();
663       }
664       tot_mgr_missing_state_in_delete.incrementAndGet();
665       LOG.debug("deleted task without in memory state " + path);
666       return;
667     }
668     synchronized (task) {
669       task.status = DELETED;
670       task.notify();
671     }
672     tot_mgr_task_deleted.incrementAndGet();
673   }
674 
675   private void deleteNodeFailure(String path) {
676     LOG.info("Failed to delete node " + path + " and will retry soon.");
677     return;
678   }
679 
680   /**
681    * signal the workers that a task was resubmitted by creating the
682    * RESCAN node.
683    * @throws KeeperException 
684    */
685   private void createRescanNode(long retries) {
686     // The RESCAN node will be deleted almost immediately by the
687     // SplitLogManager as soon as it is created because it is being
688     // created in the DONE state. This behavior prevents a buildup
689     // of RESCAN nodes. But there is also a chance that a SplitLogWorker
690     // might miss the watch-trigger that creation of RESCAN node provides.
691     // Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
692     // therefore this behavior is safe.
693     this.watcher.getRecoverableZooKeeper().getZooKeeper().
694       create(ZKSplitLog.getRescanNode(watcher),
695         TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE,
696         CreateMode.EPHEMERAL_SEQUENTIAL,
697         new CreateRescanAsyncCallback(), Long.valueOf(retries));
698   }
699 
700   private void createRescanSuccess(String path) {
701     lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
702     tot_mgr_rescan.incrementAndGet();
703     getDataSetWatch(path, zkretries);
704   }
705 
706   private void createRescanFailure() {
707     LOG.fatal("logic failure, rescan failure must not happen");
708   }
709 
710   /**
711    * @param path
712    * @param batch
713    * @return null on success, existing task on error
714    */
715   private Task createTaskIfAbsent(String path, TaskBatch batch) {
716     Task oldtask;
717     // batch.installed is only changed via this function and
718     // a single thread touches batch.installed.
719     Task newtask = new Task();
720     newtask.batch = batch;
721     oldtask = tasks.putIfAbsent(path, newtask);
722     if (oldtask == null) {
723       batch.installed++;
724       return  null;
725     }
726     // new task was not used.
727     synchronized (oldtask) {
728       if (oldtask.isOrphan()) {
729         if (oldtask.status == SUCCESS) {
730           // The task is already done. Do not install the batch for this
731           // task because it might be too late for setDone() to update
732           // batch.done. There is no need for the batch creator to wait for
733           // this task to complete.
734           return (null);
735         }
736         if (oldtask.status == IN_PROGRESS) {
737           oldtask.batch = batch;
738           batch.installed++;
739           LOG.debug("Previously orphan task " + path +
740               " is now being waited upon");
741           return null;
742         }
743         while (oldtask.status == FAILURE) {
744           LOG.debug("wait for status of task " + path +
745               " to change to DELETED");
746           tot_mgr_wait_for_zk_delete.incrementAndGet();
747           try {
748             oldtask.wait();
749           } catch (InterruptedException e) {
750             Thread.currentThread().interrupt();
751             LOG.warn("Interrupted when waiting for znode delete callback");
752             // fall through to return failure
753             break;
754           }
755         }
756         if (oldtask.status != DELETED) {
757           LOG.warn("Failure because previously failed task" +
758               " state still present. Waiting for znode delete callback" +
759               " path=" + path);
760           return oldtask;
761         }
762         // reinsert the newTask and it must succeed this time
763         Task t = tasks.putIfAbsent(path, newtask);
764         if (t == null) {
765           batch.installed++;
766           return  null;
767         }
768         LOG.fatal("Logic error. Deleted task still present in tasks map");
769         assert false : "Deleted task still present in tasks map";
770         return t;
771       }
772       LOG.warn("Failure because two threads can't wait for the same task. " +
773           " path=" + path);
774       return oldtask;
775     }
776   }
777 
778   Task findOrCreateOrphanTask(String path) {
779     Task orphanTask = new Task();
780     Task task;
781     task = tasks.putIfAbsent(path, orphanTask);
782     if (task == null) {
783       LOG.info("creating orphan task " + path);
784       tot_mgr_orphan_task_acquired.incrementAndGet();
785       task = orphanTask;
786     }
787     return task;
788   }
789 
790   @Override
791   public void nodeDataChanged(String path) {
792     Task task;
793     task = tasks.get(path);
794     if (task != null || ZKSplitLog.isRescanNode(watcher, path)) {
795       if (task != null) {
796         task.heartbeatNoDetails(EnvironmentEdgeManager.currentTimeMillis());
797       }
798       getDataSetWatch(path, zkretries);
799     }
800   }
801 
802   public void stop() {
803     if (timeoutMonitor != null) {
804       timeoutMonitor.interrupt();
805     }
806   }
807 
808   private void lookForOrphans() {
809     List<String> orphans;
810     try {
811        orphans = ZKUtil.listChildrenNoWatch(this.watcher,
812           this.watcher.splitLogZNode);
813       if (orphans == null) {
814         LOG.warn("could not get children of " + this.watcher.splitLogZNode);
815         return;
816       }
817     } catch (KeeperException e) {
818       LOG.warn("could not get children of " + this.watcher.splitLogZNode +
819           " " + StringUtils.stringifyException(e));
820       return;
821     }
822     int rescan_nodes = 0;
823     for (String path : orphans) {
824       String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
825       if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
826         rescan_nodes++;
827         LOG.debug("found orphan rescan node " + path);
828       } else {
829         LOG.info("found orphan task " + path);
830       }
831       getDataSetWatch(nodepath, zkretries);
832     }
833     LOG.info("found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
834         rescan_nodes + " rescan nodes");
835   }
836 
837   /**
838    * Keeps track of the batch of tasks submitted together by a caller in
839    * splitLogDistributed(). Clients threads use this object to wait for all
840    * their tasks to be done.
841    * <p>
842    * All access is synchronized.
843    */
844   static class TaskBatch {
845     int installed = 0;
846     int done = 0;
847     int error = 0;
848     volatile boolean isDead = false;
849 
850     @Override
851     public String toString() {
852       return ("installed = " + installed + " done = " + done + " error = "
853           + error);
854     }
855   }
856 
857   /**
858    * in memory state of an active task.
859    */
860   static class Task {
861     volatile long last_update;
862     volatile int last_version;
863     volatile String cur_worker_name;
864     TaskBatch batch;
865     volatile TerminationStatus status;
866     volatile int incarnation;
867     volatile int unforcedResubmits;
868     volatile boolean resubmitThresholdReached;
869 
870     @Override
871     public String toString() {
872       return ("last_update = " + last_update +
873           " last_version = " + last_version +
874           " cur_worker_name = " + cur_worker_name +
875           " status = " + status +
876           " incarnation = " + incarnation +
877           " resubmits = " + unforcedResubmits +
878           " batch = " + batch);
879     }
880 
881     Task() {
882       incarnation = 0;
883       last_version = -1;
884       status = IN_PROGRESS;
885       setUnassigned();
886     }
887 
888     public boolean isOrphan() {
889       return (batch == null || batch.isDead);
890     }
891 
892     public boolean isUnassigned() {
893       return (cur_worker_name == null);
894     }
895 
896     public void heartbeatNoDetails(long time) {
897       last_update = time;
898     }
899 
900     public void heartbeat(long time, int version, String worker) {
901       last_version = version;
902       last_update = time;
903       cur_worker_name = worker;
904     }
905 
906     public void setUnassigned() {
907       cur_worker_name = null;
908       last_update = -1;
909     }
910   }
911 
912   void handleDeadWorker(String workerName) {
913     // resubmit the tasks on the TimeoutMonitor thread. Makes it easier
914     // to reason about concurrency. Makes it easier to retry.
915     synchronized (deadWorkersLock) {
916       if (deadWorkers == null) {
917         deadWorkers = new HashSet<String>(100);
918       }
919       deadWorkers.add(workerName);
920     }
921     LOG.info("dead splitlog worker " + workerName);
922   }
923 
924   void handleDeadWorkers(List<ServerName> serverNames) {
925     List<String> workerNames = new ArrayList<String>(serverNames.size());
926     for (ServerName serverName : serverNames) {
927       workerNames.add(serverName.toString());
928     }
929     synchronized (deadWorkersLock) {
930       if (deadWorkers == null) {
931         deadWorkers = new HashSet<String>(100);
932       }
933       deadWorkers.addAll(workerNames);
934     }
935     LOG.info("dead splitlog workers " + workerNames);
936   }
937 
938   /**
939    * Periodically checks all active tasks and resubmits the ones that have timed
940    * out
941    */
942   private class TimeoutMonitor extends Chore {
943     public TimeoutMonitor(final int period, Stoppable stopper) {
944       super("SplitLogManager Timeout Monitor", period, stopper);
945     }
946 
947     @Override
948     protected void chore() {
949       int resubmitted = 0;
950       int unassigned = 0;
951       int tot = 0;
952       boolean found_assigned_task = false;
953       Set<String> localDeadWorkers;
954 
955       synchronized (deadWorkersLock) {
956         localDeadWorkers = deadWorkers;
957         deadWorkers = null;
958       }
959 
960       for (Map.Entry<String, Task> e : tasks.entrySet()) {
961         String path = e.getKey();
962         Task task = e.getValue();
963         String cur_worker = task.cur_worker_name;
964         tot++;
965         // don't easily resubmit a task which hasn't been picked up yet. It
966         // might be a long while before a SplitLogWorker is free to pick up a
967         // task. This is because a SplitLogWorker picks up a task one at a
968         // time. If we want progress when there are no region servers then we
969         // will have to run a SplitLogWorker thread in the Master.
970         if (task.isUnassigned()) {
971           unassigned++;
972           continue;
973         }
974         found_assigned_task = true;
975         if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
976           tot_mgr_resubmit_dead_server_task.incrementAndGet();
977           if (resubmit(path, task, FORCE)) {
978             resubmitted++;
979           } else {
980             handleDeadWorker(cur_worker);
981             LOG.warn("Failed to resubmit task " + path + " owned by dead " +
982                 cur_worker + ", will retry.");
983           }
984         } else if (resubmit(path, task, CHECK)) {
985           resubmitted++;
986         }
987       }
988       if (tot > 0) {
989         LOG.debug("total tasks = " + tot + " unassigned = " + unassigned);
990       }
991       if (resubmitted > 0) {
992         LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
993       }
994       // If there are pending tasks and all of them have been unassigned for
995       // some time then put up a RESCAN node to ping the workers.
996       // ZKSplitlog.DEFAULT_UNASSIGNED_TIMEOUT is of the order of minutes
997       // because a. it is very unlikely that every worker had a
998       // transient error when trying to grab the task b. if there are no
999       // workers then all tasks wills stay unassigned indefinitely and the
1000       // manager will be indefinitely creating RESCAN nodes. TODO may be the
1001       // master should spawn both a manager and a worker thread to guarantee
1002       // that there is always one worker in the system
1003       if (tot > 0 && !found_assigned_task &&
1004           ((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
1005           unassignedTimeout)) {
1006         for (Map.Entry<String, Task> e : tasks.entrySet()) {
1007           String path = e.getKey();
1008           Task task = e.getValue();
1009           // we have to do task.isUnassigned() check again because tasks might
1010           // have been asynchronously assigned. There is no locking required
1011           // for these checks ... it is OK even if tryGetDataSetWatch() is
1012           // called unnecessarily for a task
1013           if (task.isUnassigned() && (task.status != FAILURE)) {
1014             // We just touch the znode to make sure its still there
1015             tryGetDataSetWatch(path);
1016           }
1017         }
1018         createRescanNode(Long.MAX_VALUE);
1019         tot_mgr_resubmit_unassigned.incrementAndGet();
1020         LOG.debug("resubmitting unassigned task(s) after timeout");
1021       }
1022 
1023       // Retry previously failed deletes
1024       if (failedDeletions.size() > 0) {
1025         List<String> tmpPaths = new ArrayList<String>(failedDeletions);
1026         failedDeletions.removeAll(tmpPaths);
1027         for (String tmpPath : tmpPaths) {
1028           // deleteNode is an async call
1029           deleteNode(tmpPath, zkretries);
1030         }
1031       }
1032     }
1033   }
1034 
1035   /**
1036    * Asynchronous handler for zk create node results.
1037    * Retries on failures.
1038    */
1039   class CreateAsyncCallback implements AsyncCallback.StringCallback {
1040     private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
1041 
1042     @Override
1043     public void processResult(int rc, String path, Object ctx, String name) {
1044       tot_mgr_node_create_result.incrementAndGet();
1045       if (rc != 0) {
1046         if (shouldAbandonRetries(rc, "Create znode " + path)) {
1047           createNodeFailure(path);
1048           return;
1049         }
1050         if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
1051           // What if there is a delete pending against this pre-existing
1052           // znode? Then this soon-to-be-deleted task znode must be in TASK_DONE
1053           // state. Only operations that will be carried out on this node by
1054           // this manager are get-znode-data, task-finisher and delete-znode.
1055           // And all code pieces correctly handle the case of suddenly
1056           // disappearing task-znode.
1057           LOG.debug("found pre-existing znode " + path);
1058           tot_mgr_node_already_exists.incrementAndGet();
1059         } else {
1060           Long retry_count = (Long)ctx;
1061           LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
1062               path + " remaining retries=" + retry_count);
1063           if (retry_count == 0) {
1064             tot_mgr_node_create_err.incrementAndGet();
1065             createNodeFailure(path);
1066           } else {
1067             tot_mgr_node_create_retry.incrementAndGet();
1068             createNode(path, retry_count - 1);
1069           }
1070           return;
1071         }
1072       }
1073       createNodeSuccess(path);
1074     }
1075   }
1076 
1077   /**
1078    * Asynchronous handler for zk get-data-set-watch on node results.
1079    * Retries on failures.
1080    */
1081   class GetDataAsyncCallback implements AsyncCallback.DataCallback {
1082     private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
1083 
1084     @Override
1085     public void processResult(int rc, String path, Object ctx, byte[] data,
1086         Stat stat) {
1087       tot_mgr_get_data_result.incrementAndGet();
1088       if (rc != 0) {
1089         if (shouldAbandonRetries(rc, "GetData from znode " + path)) {
1090           return;
1091         }
1092         if (rc == KeeperException.Code.NONODE.intValue()) {
1093           tot_mgr_get_data_nonode.incrementAndGet();
1094           // The task znode has been deleted. Must be some pending delete
1095           // that deleted the task. Assume success because a task-znode is
1096           // is only deleted after TaskFinisher is successful.
1097           LOG.warn("task znode " + path + " vanished.");
1098           getDataSetWatchSuccess(path, null, Integer.MIN_VALUE);
1099           return;
1100         }
1101         Long retry_count = (Long) ctx;
1102 
1103         if (retry_count < 0) {
1104           LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1105               path + ". Ignoring error. No error handling. No retrying.");
1106           return;
1107         }
1108         LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
1109             path + " remaining retries=" + retry_count);
1110         if (retry_count == 0) {
1111           tot_mgr_get_data_err.incrementAndGet();
1112           getDataSetWatchFailure(path);
1113         } else {
1114           tot_mgr_get_data_retry.incrementAndGet();
1115           getDataSetWatch(path, retry_count - 1);
1116         }
1117         return;
1118       }
1119       getDataSetWatchSuccess(path, data, stat.getVersion());
1120       return;
1121     }
1122   }
1123 
1124   /**
1125    * Asynchronous handler for zk delete node results.
1126    * Retries on failures.
1127    */
1128   class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
1129     private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
1130 
1131     @Override
1132     public void processResult(int rc, String path, Object ctx) {
1133       tot_mgr_node_delete_result.incrementAndGet();
1134       if (rc != 0) {
1135         if (shouldAbandonRetries(rc, "Delete znode " + path)) {
1136           failedDeletions.add(path);
1137           return;
1138         }
1139         if (rc != KeeperException.Code.NONODE.intValue()) {
1140           tot_mgr_node_delete_err.incrementAndGet();
1141           Long retry_count = (Long) ctx;
1142           LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
1143               path + " remaining retries=" + retry_count);
1144           if (retry_count == 0) {
1145             LOG.warn("delete failed " + path);
1146             failedDeletions.add(path);
1147             deleteNodeFailure(path);
1148           } else {
1149             deleteNode(path, retry_count - 1);
1150           }
1151           return;
1152         } else {
1153         LOG.debug(path +
1154             " does not exist. Either was created but deleted behind our" +
1155             " back by another pending delete OR was deleted" +
1156             " in earlier retry rounds. zkretries = " + (Long) ctx);
1157         }
1158       } else {
1159         LOG.debug("deleted " + path);
1160       }
1161       deleteNodeSuccess(path);
1162     }
1163   }
1164 
1165   /**
1166    * Asynchronous handler for zk create RESCAN-node results.
1167    * Retries on failures.
1168    * <p>
1169    * A RESCAN node is created using PERSISTENT_SEQUENTIAL flag. It is a signal
1170    * for all the {@link SplitLogWorker}s to rescan for new tasks.
1171    */
1172   class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
1173     private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
1174 
1175     @Override
1176     public void processResult(int rc, String path, Object ctx, String name) {
1177       if (rc != 0) {
1178         if (shouldAbandonRetries(rc, "CreateRescan znode " + path)) {
1179           return;
1180         }
1181         Long retry_count = (Long)ctx;
1182         LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
1183             " remaining retries=" + retry_count);
1184         if (retry_count == 0) {
1185           createRescanFailure();
1186         } else {
1187           createRescanNode(retry_count - 1);
1188         }
1189         return;
1190       }
1191       // path is the original arg, name is the actual name that was created
1192       createRescanSuccess(name);
1193     }
1194   }
1195 
1196   /**
1197    * {@link SplitLogManager} can use objects implementing this interface to
1198    * finish off a partially done task by {@link SplitLogWorker}. This provides
1199    * a serialization point at the end of the task processing. Must be
1200    * restartable and idempotent.
1201    */
1202   static public interface TaskFinisher {
1203     /**
1204      * status that can be returned finish()
1205      */
1206     static public enum Status {
1207       /**
1208        * task completed successfully
1209        */
1210       DONE(),
1211       /**
1212        * task completed with error
1213        */
1214       ERR();
1215     }
1216     /**
1217      * finish the partially done task. workername provides clue to where the
1218      * partial results of the partially done tasks are present. taskname is the
1219      * name of the task that was put up in zookeeper.
1220      * <p>
1221      * @param workerName
1222      * @param taskname
1223      * @return DONE if task completed successfully, ERR otherwise
1224      */
1225     public Status finish(String workerName, String taskname);
1226   }
1227   enum ResubmitDirective {
1228     CHECK(),
1229     FORCE();
1230   }
1231   enum TerminationStatus {
1232     IN_PROGRESS("in_progress"),
1233     SUCCESS("success"),
1234     FAILURE("failure"),
1235     DELETED("deleted");
1236 
1237     String statusMsg;
1238     TerminationStatus(String msg) {
1239       statusMsg = msg;
1240     }
1241     
1242     @Override
1243     public String toString() {
1244       return statusMsg;
1245     }
1246   }
1247   
1248   /**
1249    * Completes the initialization
1250    */
1251   public void finishInitialization() {
1252     finishInitialization(false);
1253   }
1254 }