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.HashSet;
27  import java.util.Set;
28  import java.util.List;
29  import java.util.Map;
30  import java.util.concurrent.ConcurrentHashMap;
31  import java.util.concurrent.ConcurrentMap;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.FileStatus;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.hbase.Chore;
40  import org.apache.hadoop.hbase.Stoppable;
41  import org.apache.hadoop.hbase.master.SplitLogManager.TaskFinisher.Status;
42  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
43  import org.apache.hadoop.hbase.monitoring.TaskMonitor;
44  import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
45  import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
46  import org.apache.hadoop.hbase.regionserver.wal.OrphanHLogAfterSplitException;
47  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
48  import org.apache.hadoop.hbase.util.FSUtils;
49  import org.apache.hadoop.hbase.util.Threads;
50  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
51  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
52  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
53  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
54  import org.apache.hadoop.util.StringUtils;
55  import org.apache.zookeeper.AsyncCallback;
56  import org.apache.zookeeper.CreateMode;
57  import org.apache.zookeeper.KeeperException;
58  import org.apache.zookeeper.KeeperException.NoNodeException;
59  import org.apache.zookeeper.ZooDefs.Ids;
60  import org.apache.zookeeper.data.Stat;
61  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
62  
63  import static org.apache.hadoop.hbase.master.SplitLogManager.ResubmitDirective.*;
64  import static org.apache.hadoop.hbase.master.SplitLogManager.TerminationStatus.*;
65  
66  /**
67   * Distributes the task of log splitting to the available region servers.
68   * Coordination happens via zookeeper. For every log file that has to be split a
69   * znode is created under /hbase/splitlog. SplitLogWorkers race to grab a task.
70   *
71   * SplitLogManager monitors the task znodes that it creates using the
72   * timeoutMonitor thread. If a task's progress is slow then
73   * resubmit(String, boolean) will take away the task from the owner
74   * {@link SplitLogWorker} and the task will be
75   * upforgrabs again. When the task is done then the task's znode is deleted by
76   * SplitLogManager.
77   *
78   * Clients call {@link #splitLogDistributed(Path)} to split a region server's
79   * log files. The caller thread waits in this method until all the log files
80   * have been split.
81   *
82   * All the zookeeper calls made by this class are asynchronous. This is mainly
83   * to help reduce response time seen by the callers.
84   *
85   * There is race in this design between the SplitLogManager and the
86   * SplitLogWorker. SplitLogManager might re-queue a task that has in reality
87   * already been completed by a SplitLogWorker. We rely on the idempotency of
88   * the log splitting task for correctness.
89   *
90   * It is also assumed that every log splitting task is unique and once
91   * completed (either with success or with error) it will be not be submitted
92   * again. If a task is resubmitted then there is a risk that old "delete task"
93   * can delete the re-submission.
94   */
95  public class SplitLogManager extends ZooKeeperListener {
96    private static final Log LOG = LogFactory.getLog(SplitLogManager.class);
97  
98    private final Stoppable stopper;
99    private final String serverName;
100   private final TaskFinisher taskFinisher;
101   private FileSystem fs;
102   private Configuration conf;
103 
104   private long zkretries;
105   private long resubmit_threshold;
106   private long timeout;
107   private long unassignedTimeout;
108   private long lastNodeCreateTime = Long.MAX_VALUE;
109 
110   private ConcurrentMap<String, Task> tasks =
111     new ConcurrentHashMap<String, Task>();
112   private TimeoutMonitor timeoutMonitor;
113 
114   private Set<String> deadWorkers = null;
115   private Object deadWorkersLock = new Object();
116 
117   /**
118    * Its OK to construct this object even when region-servers are not online. It
119    * does lookup the orphan tasks in zk but it doesn't block for them to be
120    * done.
121    *
122    * @param zkw
123    * @param conf
124    * @param stopper
125    * @param serverName
126    */
127   public SplitLogManager(ZooKeeperWatcher zkw, final Configuration conf,
128       Stoppable stopper, String serverName) {
129     this(zkw, conf, stopper, serverName, new TaskFinisher() {
130       @Override
131       public Status finish(String workerName, String logfile) {
132         String tmpname =
133           ZKSplitLog.getSplitLogDirTmpComponent(workerName, logfile);
134         try {
135           HLogSplitter.moveRecoveredEditsFromTemp(tmpname, logfile, conf);
136         } catch (IOException e) {
137           LOG.warn("Could not finish splitting of log file " + logfile);
138           return Status.ERR;
139         }
140         return Status.DONE;
141       }
142     });
143   }
144   public SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
145       Stoppable stopper, String serverName, TaskFinisher tf) {
146     super(zkw);
147     this.taskFinisher = tf;
148     this.conf = conf;
149     this.stopper = stopper;
150     this.zkretries = conf.getLong("hbase.splitlog.zk.retries",
151         ZKSplitLog.DEFAULT_ZK_RETRIES);
152     this.resubmit_threshold = conf.getLong("hbase.splitlog.max.resubmit",
153         ZKSplitLog.DEFAULT_MAX_RESUBMIT);
154     this.timeout = conf.getInt("hbase.splitlog.manager.timeout",
155         ZKSplitLog.DEFAULT_TIMEOUT);
156     this.unassignedTimeout =
157       conf.getInt("hbase.splitlog.manager.unassigned.timeout",
158         ZKSplitLog.DEFAULT_UNASSIGNED_TIMEOUT);
159     LOG.debug("timeout = " + timeout);
160     LOG.debug("unassigned timeout = " + unassignedTimeout);
161 
162     this.serverName = serverName;
163     this.timeoutMonitor = new TimeoutMonitor(
164         conf.getInt("hbase.splitlog.manager.timeoutmonitor.period",
165             1000),
166         stopper);
167   }
168 
169   public void finishInitialization() {
170     Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), serverName +
171       ".splitLogManagerTimeoutMonitor");
172     // Watcher can be null during tests with Mock'd servers.
173     if (this.watcher != null) {
174       this.watcher.registerListener(this);
175       lookForOrphans();
176     }
177   }
178 
179   private FileStatus[] getFileList(List<Path> logDirs) throws IOException {
180     List<FileStatus> fileStatus = new ArrayList<FileStatus>();
181     for (Path hLogDir : logDirs) {
182       this.fs = hLogDir.getFileSystem(conf);
183       if (!fs.exists(hLogDir)) {
184         LOG.warn(hLogDir + " doesn't exist. Nothing to do!");
185         continue;
186       }
187       // TODO filter filenames?
188       FileStatus[] logfiles = FSUtils.listStatus(fs, hLogDir, null);
189       if (logfiles == null || logfiles.length == 0) {
190         LOG.info(hLogDir + " is empty dir, no logs to split");
191       } else {
192         for (FileStatus status : logfiles)
193           fileStatus.add(status);
194       }
195     }
196     if (fileStatus.isEmpty())
197       return null;
198     FileStatus[] a = new FileStatus[fileStatus.size()];
199     return fileStatus.toArray(a);
200   }
201 
202   /**
203    * @param logDir
204    *            one region sever hlog dir path in .logs
205    * @throws IOException
206    *             if there was an error while splitting any log file
207    * @return cumulative size of the logfiles split
208    * @throws IOException 
209    */
210   public long splitLogDistributed(final Path logDir) throws IOException {
211     List<Path> logDirs = new ArrayList<Path>();
212     logDirs.add(logDir);
213     return splitLogDistributed(logDirs);
214   }
215   /**
216    * The caller will block until all the log files of the given region server
217    * have been processed - successfully split or an error is encountered - by an
218    * available worker region server. This method must only be called after the
219    * region servers have been brought online.
220    *
221    * @param logDirs
222    * @throws IOException
223    *          if there was an error while splitting any log file
224    * @return cumulative size of the logfiles split
225    */
226   public long splitLogDistributed(final List<Path> logDirs) throws IOException {
227     MonitoredTask status = TaskMonitor.get().createStatus(
228           "Doing distributed log split in " + logDirs);
229     FileStatus[] logfiles = getFileList(logDirs);
230     if(logfiles == null)
231       return 0;
232     status.setStatus("Checking directory contents...");
233     LOG.debug("Scheduling batch of logs to split");
234     tot_mgr_log_split_batch_start.incrementAndGet();
235     LOG.info("started splitting logs in " + logDirs);
236     long t = EnvironmentEdgeManager.currentTimeMillis();
237     long totalSize = 0;
238     TaskBatch batch = new TaskBatch();
239     for (FileStatus lf : logfiles) {
240       // TODO If the log file is still being written to - which is most likely
241       // the case for the last log file - then its length will show up here
242       // as zero. The size of such a file can only be retrieved after
243       // recover-lease is done. totalSize will be under in most cases and the
244       // metrics that it drives will also be under-reported.
245       totalSize += lf.getLen();
246       if (installTask(lf.getPath().toString(), batch) == false) {
247         throw new IOException("duplicate log split scheduled for "
248             + lf.getPath());
249       }
250     }
251     waitTasks(batch, status);
252     if (batch.done != batch.installed) {
253       stopTrackingTasks(batch);
254       tot_mgr_log_split_batch_err.incrementAndGet();
255       LOG.warn("error while splitting logs in " + logDirs +
256       " installed = " + batch.installed + " but only " + batch.done + " done");
257       throw new IOException("error or interrupt while splitting logs in "
258           + logDirs + " Task = " + batch);
259     }
260     for(Path logDir: logDirs){
261       if (anyNewLogFiles(logDir, logfiles)) {
262         tot_mgr_new_unexpected_hlogs.incrementAndGet();
263         LOG.warn("new hlogs were produced while logs in " + logDir +
264           " were being split");
265         throw new OrphanHLogAfterSplitException();
266       }
267       tot_mgr_log_split_batch_success.incrementAndGet();
268       status.setStatus("Cleaning up log directory...");
269       if (!fs.delete(logDir, true)) {
270         throw new IOException("Unable to delete src dir: " + logDir);
271       }
272     }
273     String msg = "finished splitting (more than or equal to) " + totalSize +
274         " bytes in " + batch.installed + " log files in " + logDirs + " in " +
275         (EnvironmentEdgeManager.currentTimeMillis() - t) + "ms";
276     status.markComplete(msg);
277     LOG.info(msg);
278     return totalSize;
279   }
280 
281   boolean installTask(String taskname, TaskBatch batch) {
282     tot_mgr_log_split_start.incrementAndGet();
283     String path = ZKSplitLog.getEncodedNodeName(watcher, taskname);
284     Task oldtask = createTaskIfAbsent(path, batch);
285     if (oldtask == null) {
286       // publish the task in zk
287       createNode(path, zkretries);
288       return true;
289     }
290     LOG.warn(path + "is already being split. " +
291         "Two threads cannot wait for the same task");
292     return false;
293   }
294 
295   private void waitTasks(TaskBatch batch, MonitoredTask status) {
296     synchronized (batch) {
297       while ((batch.done + batch.error) != batch.installed) {
298         try {
299           status.setStatus("Waiting for distributed tasks to finish. "
300               + " scheduled=" + batch.installed
301               + " done=" + batch.done
302               + " error=" + batch.error);
303           batch.wait(100);
304           if (stopper.isStopped()) {
305             LOG.warn("Stopped while waiting for log splits to be completed");
306             return;
307           }
308         } catch (InterruptedException e) {
309           LOG.warn("Interrupted while waiting for log splits to be completed");
310           Thread.currentThread().interrupt();
311           return;
312         }
313       }
314     }
315   }
316 
317   private void setDone(String path, TerminationStatus status) {
318     if (!ZKSplitLog.isRescanNode(watcher, path)) {
319       if (status == SUCCESS) {
320         tot_mgr_log_split_success.incrementAndGet();
321         LOG.info("Done splitting " + path);
322       } else {
323         tot_mgr_log_split_err.incrementAndGet();
324         LOG.warn("Error splitting " + path);
325       }
326     }
327     Task task = tasks.get(path);
328     if (task == null) {
329       if (!ZKSplitLog.isRescanNode(watcher, path)) {
330         tot_mgr_unacquired_orphan_done.incrementAndGet();
331         LOG.debug("unacquired orphan task is done " + path);
332       }
333     } else {
334       // if in stopTrackingTasks() we were to make tasks orphan instead of
335       // forgetting about them then we will have to handle the race when
336       // accessing task.batch here.
337       if (!task.isOrphan()) {
338         synchronized (task.batch) {
339           if (status == SUCCESS) {
340             task.batch.done++;
341           } else {
342             task.batch.error++;
343           }
344           if ((task.batch.done + task.batch.error) == task.batch.installed) {
345             task.batch.notify();
346           }
347         }
348       }
349       task.deleted = true;
350     }
351     // delete the task node in zk. Keep trying indefinitely - its an async
352     // call and no one is blocked waiting for this node to be deleted. All
353     // task names are unique (log.<timestamp>) there is no risk of deleting
354     // a future task.
355     deleteNode(path, Long.MAX_VALUE);
356     return;
357   }
358 
359   private void createNode(String path, Long retry_count) {
360     ZKUtil.asyncCreate(this.watcher, path,
361         TaskState.TASK_UNASSIGNED.get(serverName), new CreateAsyncCallback(),
362         retry_count);
363     tot_mgr_node_create_queued.incrementAndGet();
364     return;
365   }
366 
367   private void createNodeSuccess(String path) {
368     lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
369     LOG.debug("put up splitlog task at znode " + path);
370     getDataSetWatch(path, zkretries);
371   }
372 
373   private void createNodeFailure(String path) {
374     // TODO the Manger should split the log locally instead of giving up
375     LOG.warn("failed to create task node" + path);
376     setDone(path, FAILURE);
377   }
378 
379 
380   private void getDataSetWatch(String path, Long retry_count) {
381     this.watcher.getRecoverableZooKeeper().getZooKeeper().
382         getData(path, this.watcher,
383         new GetDataAsyncCallback(), retry_count);
384     tot_mgr_get_data_queued.incrementAndGet();
385   }
386 
387   private void getDataSetWatchSuccess(String path, byte[] data, int version) {
388     if (data == null) {
389       tot_mgr_null_data.incrementAndGet();
390       LOG.fatal("logic error - got null data " + path);
391       setDone(path, FAILURE);
392       return;
393     }
394     data = this.watcher.getRecoverableZooKeeper().removeMetaData(data);
395     // LOG.debug("set watch on " + path + " got data " + new String(data));
396     if (TaskState.TASK_UNASSIGNED.equals(data)) {
397       LOG.debug("task not yet acquired " + path + " ver = " + version);
398       handleUnassignedTask(path);
399     } else if (TaskState.TASK_OWNED.equals(data)) {
400       heartbeat(path, version,
401           TaskState.TASK_OWNED.getWriterName(data));
402     } else if (TaskState.TASK_RESIGNED.equals(data)) {
403       LOG.info("task " + path + " entered state " + new String(data));
404       resubmitOrFail(path, FORCE);
405     } else if (TaskState.TASK_DONE.equals(data)) {
406       LOG.info("task " + path + " entered state " + new String(data));
407       if (taskFinisher != null && !ZKSplitLog.isRescanNode(watcher, path)) {
408         if (taskFinisher.finish(TaskState.TASK_DONE.getWriterName(data),
409             ZKSplitLog.getFileName(path)) == Status.DONE) {
410           setDone(path, SUCCESS);
411         } else {
412           resubmitOrFail(path, CHECK);
413         }
414       } else {
415         setDone(path, SUCCESS);
416       }
417     } else if (TaskState.TASK_ERR.equals(data)) {
418       LOG.info("task " + path + " entered state " + new String(data));
419       resubmitOrFail(path, CHECK);
420     } else {
421       LOG.fatal("logic error - unexpected zk state for path = " + path
422           + " data = " + new String(data));
423       setDone(path, FAILURE);
424     }
425   }
426 
427   private void getDataSetWatchFailure(String path) {
428     LOG.warn("failed to set data watch " + path);
429     setDone(path, FAILURE);
430   }
431 
432   /**
433    * It is possible for a task to stay in UNASSIGNED state indefinitely - say
434    * SplitLogManager wants to resubmit a task. It forces the task to UNASSIGNED
435    * state but it dies before it could create the RESCAN task node to signal
436    * the SplitLogWorkers to pick up the task. To prevent this scenario the
437    * SplitLogManager resubmits all orphan and UNASSIGNED tasks at startup.
438    *
439    * @param path
440    */
441   private void handleUnassignedTask(String path) {
442     if (ZKSplitLog.isRescanNode(watcher, path)) {
443       return;
444     }
445     Task task = findOrCreateOrphanTask(path);
446     if (task.isOrphan() && (task.incarnation == 0)) {
447       LOG.info("resubmitting unassigned orphan task " + path);
448       // ignore failure to resubmit. The timeout-monitor will handle it later
449       // albeit in a more crude fashion
450       resubmit(path, task, FORCE);
451     }
452   }
453 
454   private void heartbeat(String path, int new_version,
455       String workerName) {
456     Task task = findOrCreateOrphanTask(path);
457     if (new_version != task.last_version) {
458       if (task.isUnassigned()) {
459         LOG.info("task " + path + " acquired by " + workerName);
460       }
461       task.heartbeat(EnvironmentEdgeManager.currentTimeMillis(),
462           new_version, workerName);
463       tot_mgr_heartbeat.incrementAndGet();
464     } else {
465       assert false;
466       LOG.warn("got dup heartbeat for " + path + " ver = " + new_version);
467     }
468     return;
469   }
470 
471   private boolean resubmit(String path, Task task,
472       ResubmitDirective directive) {
473     // its ok if this thread misses the update to task.deleted. It will
474     // fail later
475     if (task.deleted) {
476       return false;
477     }
478     int version;
479     if (directive != FORCE) {
480       if ((EnvironmentEdgeManager.currentTimeMillis() - task.last_update) <
481           timeout) {
482         return false;
483       }
484       if (task.unforcedResubmits >= resubmit_threshold) {
485         if (task.unforcedResubmits == resubmit_threshold) {
486           tot_mgr_resubmit_threshold_reached.incrementAndGet();
487           LOG.info("Skipping resubmissions of task " + path +
488               " because threshold " + resubmit_threshold + " reached");
489         }
490         return false;
491       }
492       // race with heartbeat() that might be changing last_version
493       version = task.last_version;
494     } else {
495       version = -1;
496     }
497     LOG.info("resubmitting task " + path);
498     task.incarnation++;
499     try {
500       // blocking zk call but this is done from the timeout thread
501       if (ZKUtil.setData(this.watcher, path,
502           TaskState.TASK_UNASSIGNED.get(serverName),
503           version) == false) {
504         LOG.debug("failed to resubmit task " + path +
505             " version changed");
506         return false;
507       }
508     } catch (NoNodeException e) {
509       LOG.debug("failed to resubmit " + path + " task done");
510       return false;
511     } catch (KeeperException e) {
512       tot_mgr_resubmit_failed.incrementAndGet();
513       LOG.warn("failed to resubmit " + path, e);
514       return false;
515     }
516     // don't count forced resubmits
517     if (directive != FORCE) {
518       task.unforcedResubmits++;
519     }
520     task.setUnassigned();
521     createRescanNode(Long.MAX_VALUE);
522     tot_mgr_resubmit.incrementAndGet();
523     return true;
524   }
525 
526   private void resubmitOrFail(String path, ResubmitDirective directive) {
527     if (resubmit(path, findOrCreateOrphanTask(path), directive) == false) {
528       setDone(path, FAILURE);
529     }
530   }
531 
532   private void deleteNode(String path, Long retries) {
533     tot_mgr_node_delete_queued.incrementAndGet();
534     this.watcher.getRecoverableZooKeeper().getZooKeeper().
535       delete(path, -1, new DeleteAsyncCallback(),
536         retries);
537   }
538 
539   private void deleteNodeSuccess(String path) {
540     Task task;
541     task = tasks.remove(path);
542     if (task == null) {
543       if (ZKSplitLog.isRescanNode(watcher, path)) {
544         tot_mgr_rescan_deleted.incrementAndGet();
545       }
546       tot_mgr_missing_state_in_delete.incrementAndGet();
547       LOG.debug("deleted task without in memory state " + path);
548       return;
549     }
550     tot_mgr_task_deleted.incrementAndGet();
551   }
552 
553   private void deleteNodeFailure(String path) {
554     LOG.fatal("logic failure, failing to delete a node should never happen " +
555         "because delete has infinite retries");
556     return;
557   }
558 
559   /**
560    * signal the workers that a task was resubmitted by creating the
561    * RESCAN node.
562    * @throws KeeperException 
563    */
564   private void createRescanNode(long retries) {
565     // The RESCAN node will be deleted almost immediately by the
566     // SplitLogManager as soon as it is created because it is being
567     // created in the DONE state. This behavior prevents a buildup
568     // of RESCAN nodes. But there is also a chance that a SplitLogWorker
569     // might miss the watch-trigger that creation of RESCAN node provides.
570     // Since the TimeoutMonitor will keep resubmitting UNASSIGNED tasks
571     // therefore this behavior is safe.
572     this.watcher.getRecoverableZooKeeper().getZooKeeper().
573       create(ZKSplitLog.getRescanNode(watcher),
574         TaskState.TASK_DONE.get(serverName), Ids.OPEN_ACL_UNSAFE,
575         CreateMode.EPHEMERAL_SEQUENTIAL,
576         new CreateRescanAsyncCallback(), new Long(retries));
577   }
578 
579   private void createRescanSuccess(String path) {
580     lastNodeCreateTime = EnvironmentEdgeManager.currentTimeMillis();
581     tot_mgr_rescan.incrementAndGet();
582     getDataSetWatch(path, zkretries);
583   }
584 
585   private void createRescanFailure() {
586     LOG.fatal("logic failure, rescan failure must not happen");
587   }
588 
589   /**
590    * @param path
591    * @param batch
592    * @return null on success, existing task on error
593    */
594   private Task createTaskIfAbsent(String path, TaskBatch batch) {
595     Task oldtask;
596     oldtask = tasks.putIfAbsent(path, new Task(batch));
597     if (oldtask != null && oldtask.isOrphan()) {
598         LOG.info("Previously orphan task " + path +
599             " is now being waited upon");
600         oldtask.setBatch(batch);
601         return (null);
602     }
603     return oldtask;
604   }
605 
606   /**
607    * This function removes any knowledge of this batch's tasks from the
608    * manager. It doesn't actually stop the active tasks. If the tasks are
609    * resubmitted then the active tasks will be reacquired and monitored by the
610    * manager. It is important to call this function when batch processing
611    * terminates prematurely, otherwise if the tasks are re-submitted
612    * then they might fail.
613    * <p>
614    * there is a slight race here. even after a task has been removed from
615    * {@link #tasks} someone who had acquired a reference to it will continue to
616    * process the task. That is OK since we don't actually change the task and
617    * the batch objects.
618    * <p>
619    * TODO Its  probably better to convert these to orphan tasks but then we
620    * have to deal with race conditions as we nullify Task's batch pointer etc.
621    * <p>
622    * @param batch
623    */
624   void stopTrackingTasks(TaskBatch batch) {
625     for (Map.Entry<String, Task> e : tasks.entrySet()) {
626       String path = e.getKey();
627       Task t = e.getValue();
628       if (t.batch == batch) { // == is correct. equals not necessary.
629         tasks.remove(path);
630       }
631     }
632   }
633 
634   Task findOrCreateOrphanTask(String path) {
635     Task orphanTask = new Task(null);
636     Task task;
637     task = tasks.putIfAbsent(path, orphanTask);
638     if (task == null) {
639       LOG.info("creating orphan task " + path);
640       tot_mgr_orphan_task_acquired.incrementAndGet();
641       task = orphanTask;
642     }
643     return task;
644   }
645 
646   @Override
647   public void nodeDataChanged(String path) {
648     if (tasks.get(path) != null || ZKSplitLog.isRescanNode(watcher, path)) {
649       getDataSetWatch(path, zkretries);
650     }
651   }
652 
653   public void stop() {
654     if (timeoutMonitor != null) {
655       timeoutMonitor.interrupt();
656     }
657   }
658 
659   private void lookForOrphans() {
660     List<String> orphans;
661     try {
662        orphans = ZKUtil.listChildrenNoWatch(this.watcher,
663           this.watcher.splitLogZNode);
664       if (orphans == null) {
665         LOG.warn("could not get children of " + this.watcher.splitLogZNode);
666         return;
667       }
668     } catch (KeeperException e) {
669       LOG.warn("could not get children of " + this.watcher.splitLogZNode +
670           " " + StringUtils.stringifyException(e));
671       return;
672     }
673     int rescan_nodes = 0;
674     for (String path : orphans) {
675       String nodepath = ZKUtil.joinZNode(watcher.splitLogZNode, path);
676       if (ZKSplitLog.isRescanNode(watcher, nodepath)) {
677         rescan_nodes++;
678         LOG.debug("found orphan rescan node " + path);
679       } else {
680         LOG.info("found orphan task " + path);
681       }
682       getDataSetWatch(nodepath, zkretries);
683     }
684     LOG.info("found " + (orphans.size() - rescan_nodes) + " orphan tasks and " +
685         rescan_nodes + " rescan nodes");
686   }
687 
688   /**
689    * Keeps track of the batch of tasks submitted together by a caller in
690    * splitLogDistributed(). Clients threads use this object to wait for all
691    * their tasks to be done.
692    * <p>
693    * All access is synchronized.
694    */
695   static class TaskBatch {
696     int installed;
697     int done;
698     int error;
699 
700     @Override
701     public String toString() {
702       return ("installed = " + installed + " done = " + done + " error = "
703           + error);
704     }
705   }
706 
707   /**
708    * in memory state of an active task.
709    */
710   static class Task {
711     long last_update;
712     int last_version;
713     String cur_worker_name;
714     TaskBatch batch;
715     boolean deleted;
716     int incarnation;
717     int unforcedResubmits;
718 
719     @Override
720     public String toString() {
721       return ("last_update = " + last_update +
722           " last_version = " + last_version +
723           " cur_worker_name = " + cur_worker_name +
724           " deleted = " + deleted +
725           " incarnation = " + incarnation +
726           " resubmits = " + unforcedResubmits +
727           " batch = " + batch);
728     }
729 
730     Task(TaskBatch tb) {
731       incarnation = 0;
732       last_version = -1;
733       deleted = false;
734       setBatch(tb);
735       setUnassigned();
736     }
737 
738     public void setBatch(TaskBatch batch) {
739       if (batch != null && this.batch != null) {
740         LOG.fatal("logic error - batch being overwritten");
741       }
742       this.batch = batch;
743       if (batch != null) {
744         if (this.incarnation == 0) {
745           batch.installed++;
746         }
747       }
748     }
749 
750     public boolean isOrphan() {
751       return (batch == null);
752     }
753 
754     public boolean isUnassigned() {
755       return (last_update == -1);
756     }
757 
758     public void heartbeat(long time, int version, String worker) {
759       last_version = version;
760       last_update = time;
761       cur_worker_name = worker;
762     }
763 
764     public void setUnassigned() {
765       cur_worker_name = null;
766       last_update = -1;
767     }
768   }
769 
770   void handleDeadWorker(String worker_name) {
771     // resubmit the tasks on the TimeoutMonitor thread. Makes it easier
772     // to reason about concurrency. Makes it easier to retry.
773     synchronized (deadWorkersLock) {
774       if (deadWorkers == null) {
775         deadWorkers = new HashSet<String>(100);
776       }
777       deadWorkers.add(worker_name);
778     }
779     LOG.info("dead splitlog worker " + worker_name);
780   }
781 
782   /**
783    * Periodically checks all active tasks and resubmits the ones that have timed
784    * out
785    */
786   private class TimeoutMonitor extends Chore {
787     public TimeoutMonitor(final int period, Stoppable stopper) {
788       super("SplitLogManager Timeout Monitor", period, stopper);
789     }
790 
791     @Override
792     protected void chore() {
793       int resubmitted = 0;
794       int unassigned = 0;
795       int tot = 0;
796       boolean found_assigned_task = false;
797       Set<String> localDeadWorkers;
798 
799       synchronized (deadWorkersLock) {
800         localDeadWorkers = deadWorkers;
801         deadWorkers = null;
802       }
803 
804       for (Map.Entry<String, Task> e : tasks.entrySet()) {
805         String path = e.getKey();
806         Task task = e.getValue();
807         String cur_worker = task.cur_worker_name;
808         tot++;
809         // don't easily resubmit a task which hasn't been picked up yet. It
810         // might be a long while before a SplitLogWorker is free to pick up a
811         // task. This is because a SplitLogWorker picks up a task one at a
812         // time. If we want progress when there are no region servers then we
813         // will have to run a SplitLogWorker thread in the Master.
814         if (task.isUnassigned()) {
815           unassigned++;
816           continue;
817         }
818         found_assigned_task = true;
819         if (localDeadWorkers != null && localDeadWorkers.contains(cur_worker)) {
820           tot_mgr_resubmit_dead_server_task.incrementAndGet();
821           if (resubmit(path, task, FORCE)) {
822             resubmitted++;
823           } else {
824             handleDeadWorker(cur_worker);
825             LOG.warn("Failed to resubmit task " + path + " owned by dead " +
826                 cur_worker + ", will retry.");
827           }
828         } else if (resubmit(path, task, CHECK)) {
829           resubmitted++;
830         }
831       }
832       if (tot > 0) {
833         LOG.debug("total tasks = " + tot + " unassigned = " + unassigned);
834       }
835       if (resubmitted > 0) {
836         LOG.info("resubmitted " + resubmitted + " out of " + tot + " tasks");
837       }
838       // If there are pending tasks and all of them have been unassigned for
839       // some time then put up a RESCAN node to ping the workers.
840       // ZKSplitlog.DEFAULT_UNASSIGNED_TIMEOUT is of the order of minutes
841       // because a. it is very unlikely that every worker had a
842       // transient error when trying to grab the task b. if there are no
843       // workers then all tasks wills stay unassigned indefinitely and the
844       // manager will be indefinitely creating RESCAN nodes. TODO may be the
845       // master should spawn both a manager and a worker thread to guarantee
846       // that there is always one worker in the system
847       if (tot > 0 && !found_assigned_task &&
848           ((EnvironmentEdgeManager.currentTimeMillis() - lastNodeCreateTime) >
849           unassignedTimeout)) {
850         createRescanNode(Long.MAX_VALUE);
851         tot_mgr_resubmit_unassigned.incrementAndGet();
852         LOG.debug("resubmitting unassigned task(s) after timeout");
853       }
854     }
855   }
856 
857   /**
858    * Asynchronous handler for zk create node results.
859    * Retries on failures.
860    */
861   class CreateAsyncCallback implements AsyncCallback.StringCallback {
862     private final Log LOG = LogFactory.getLog(CreateAsyncCallback.class);
863 
864     @Override
865     public void processResult(int rc, String path, Object ctx, String name) {
866       tot_mgr_node_create_result.incrementAndGet();
867       if (rc != 0) {
868         if (rc == KeeperException.Code.NODEEXISTS.intValue()) {
869           LOG.debug("found pre-existing znode " + path);
870           tot_mgr_node_already_exists.incrementAndGet();
871         } else {
872           Long retry_count = (Long)ctx;
873           LOG.warn("create rc =" + KeeperException.Code.get(rc) + " for " +
874               path + " retry=" + retry_count);
875           if (retry_count == 0) {
876             tot_mgr_node_create_err.incrementAndGet();
877             createNodeFailure(path);
878           } else {
879             tot_mgr_node_create_retry.incrementAndGet();
880             createNode(path, retry_count - 1);
881           }
882           return;
883         }
884       }
885       createNodeSuccess(path);
886     }
887   }
888 
889   /**
890    * Asynchronous handler for zk get-data-set-watch on node results.
891    * Retries on failures.
892    */
893   class GetDataAsyncCallback implements AsyncCallback.DataCallback {
894     private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
895 
896     @Override
897     public void processResult(int rc, String path, Object ctx, byte[] data,
898         Stat stat) {
899       tot_mgr_get_data_result.incrementAndGet();
900       if (rc != 0) {
901         Long retry_count = (Long) ctx;
902         LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " +
903             path + " retry=" + retry_count);
904         if (retry_count == 0) {
905           tot_mgr_get_data_err.incrementAndGet();
906           getDataSetWatchFailure(path);
907         } else {
908           tot_mgr_get_data_retry.incrementAndGet();
909           getDataSetWatch(path, retry_count - 1);
910         }
911         return;
912       }
913       getDataSetWatchSuccess(path, data, stat.getVersion());
914       return;
915     }
916   }
917 
918   /**
919    * Asynchronous handler for zk delete node results.
920    * Retries on failures.
921    */
922   class DeleteAsyncCallback implements AsyncCallback.VoidCallback {
923     private final Log LOG = LogFactory.getLog(DeleteAsyncCallback.class);
924 
925     @Override
926     public void processResult(int rc, String path, Object ctx) {
927       tot_mgr_node_delete_result.incrementAndGet();
928       if (rc != 0) {
929         if (rc != KeeperException.Code.NONODE.intValue()) {
930           tot_mgr_node_delete_err.incrementAndGet();
931           Long retry_count = (Long) ctx;
932           LOG.warn("delete rc=" + KeeperException.Code.get(rc) + " for " +
933               path + " retry=" + retry_count);
934           if (retry_count == 0) {
935             LOG.warn("delete failed " + path);
936             deleteNodeFailure(path);
937           } else {
938             deleteNode(path, retry_count - 1);
939           }
940           return;
941         } else {
942         LOG.debug(path
943             + " does not exist, either was never created or was deleted"
944             + " in earlier rounds, zkretries = " + (Long) ctx);
945         }
946       } else {
947         LOG.debug("deleted " + path);
948       }
949       deleteNodeSuccess(path);
950     }
951   }
952 
953   /**
954    * Asynchronous handler for zk create RESCAN-node results.
955    * Retries on failures.
956    * <p>
957    * A RESCAN node is created using PERSISTENT_SEQUENTIAL flag. It is a signal
958    * for all the {@link SplitLogWorker}s to rescan for new tasks.
959    */
960   class CreateRescanAsyncCallback implements AsyncCallback.StringCallback {
961     private final Log LOG = LogFactory.getLog(CreateRescanAsyncCallback.class);
962 
963     @Override
964     public void processResult(int rc, String path, Object ctx, String name) {
965       if (rc != 0) {
966         Long retry_count = (Long)ctx;
967         LOG.warn("rc=" + KeeperException.Code.get(rc) + " for "+ path +
968             " retry=" + retry_count);
969         if (retry_count == 0) {
970           createRescanFailure();
971         } else {
972           createRescanNode(retry_count - 1);
973         }
974         return;
975       }
976       // path is the original arg, name is the actual name that was created
977       createRescanSuccess(name);
978     }
979   }
980 
981   /**
982    * checks whether any new files have appeared in logDir which were
983    * not present in the original logfiles set
984    * @param logdir
985    * @param logfiles
986    * @return True if a new log file is found
987    * @throws IOException
988    */
989   public boolean anyNewLogFiles(Path logdir, FileStatus[] logfiles)
990   throws IOException {
991     if (logdir == null) {
992       return false;
993     }
994     LOG.debug("re-listing " + logdir);
995     tot_mgr_relist_logdir.incrementAndGet();
996     FileStatus[] newfiles = FSUtils.listStatus(fs, logdir, null);
997     if (newfiles == null) {
998       return false;
999     }
1000     boolean matched;
1001     for (FileStatus newfile : newfiles) {
1002       matched = false;
1003       for (FileStatus origfile : logfiles) {
1004         if (origfile.equals(newfile)) {
1005           matched = true;
1006           break;
1007         }
1008       }
1009       if (matched == false) {
1010         LOG.warn("Discovered orphan hlog " + newfile + " after split." +
1011         " Maybe HRegionServer was not dead when we started");
1012         return true;
1013       }
1014     }
1015     return false;
1016   }
1017 
1018   /**
1019    * {@link SplitLogManager} can use objects implementing this interface to
1020    * finish off a partially done task by {@link SplitLogWorker}. This provides
1021    * a serialization point at the end of the task processing.
1022    */
1023   static public interface TaskFinisher {
1024     /**
1025      * status that can be returned finish()
1026      */
1027     static public enum Status {
1028       /**
1029        * task completed successfully
1030        */
1031       DONE(),
1032       /**
1033        * task completed with error
1034        */
1035       ERR();
1036     }
1037     /**
1038      * finish the partially done task. workername provides clue to where the
1039      * partial results of the partially done tasks are present. taskname is the
1040      * name of the task that was put up in zookeeper.
1041      * <p>
1042      * @param workerName
1043      * @param taskname
1044      * @return DONE if task completed successfully, ERR otherwise
1045      */
1046     public Status finish(String workerName, String taskname);
1047   }
1048   enum ResubmitDirective {
1049     CHECK(),
1050     FORCE();
1051   }
1052   enum TerminationStatus {
1053     SUCCESS(),
1054     FAILURE();
1055   }
1056 }