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