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.regionserver;
21  
22  import static org.apache.hadoop.hbase.zookeeper.ZKSplitLog.Counters.*;
23  
24  import java.io.IOException;
25  import java.io.InterruptedIOException;
26  import java.util.List;
27  import java.util.concurrent.atomic.AtomicLong;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.fs.FileSystem;
33  import org.apache.hadoop.fs.Path;
34  import org.apache.hadoop.hbase.master.SplitLogManager;
35  import org.apache.hadoop.hbase.regionserver.wal.HLogSplitter;
36  import org.apache.hadoop.hbase.util.CancelableProgressable;
37  import org.apache.hadoop.hbase.util.FSUtils;
38  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
39  import org.apache.hadoop.hbase.zookeeper.ZKSplitLog.TaskState;
40  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
41  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
42  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
43  import org.apache.hadoop.util.StringUtils;
44  import org.apache.zookeeper.AsyncCallback;
45  import org.apache.zookeeper.KeeperException;
46  import org.apache.zookeeper.data.Stat;
47  
48  /**
49   * This worker is spawned in every regionserver (should we also spawn one in
50   * the master?). The Worker waits for log splitting tasks to be put up by the
51   * {@link SplitLogManager} running in the master and races with other workers
52   * in other serves to acquire those tasks. The coordination is done via
53   * zookeeper. All the action takes place at /hbase/splitlog znode.
54   * <p>
55   * If a worker has successfully moved the task from state UNASSIGNED to
56   * OWNED then it owns the task. It keeps heart beating the manager by
57   * periodically moving the task from OWNED to OWNED state. On success it
58   * moves the task to SUCCESS. On unrecoverable error it moves task state to
59   * ERR. If it cannot continue but wants the master to retry the task then it
60   * moves the task state to RESIGNED.
61   * <p>
62   * The manager can take a task away from a worker by moving the task from
63   * OWNED to UNASSIGNED. In the absence of a global lock there is a
64   * unavoidable race here - a worker might have just finished its task when it
65   * is stripped of its ownership. Here we rely on the idempotency of the log
66   * splitting task for correctness
67   */
68  public class SplitLogWorker extends ZooKeeperListener implements Runnable {
69    private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
70  
71    Thread worker;
72    private final String serverName;
73    private final TaskExecutor executor;
74    private long zkretries;
75  
76    private Object taskReadyLock = new Object();
77    volatile int taskReadySeq = 0;
78    private volatile String currentTask = null;
79    private int currentVersion;
80    private volatile boolean exitWorker;
81    private Object grabTaskLock = new Object();
82    private boolean workerInGrabTask = false;
83  
84  
85    public SplitLogWorker(ZooKeeperWatcher watcher, Configuration conf,
86        String serverName, TaskExecutor executor) {
87      super(watcher);
88      this.serverName = serverName;
89      this.executor = executor;
90      this.zkretries = conf.getLong("hbase.splitlog.zk.retries", 3);
91    }
92  
93    public SplitLogWorker(ZooKeeperWatcher watcher, final Configuration conf,
94        final String serverName) {
95      this(watcher, conf, serverName, new TaskExecutor () {
96        @Override
97        public Status exec(String filename, CancelableProgressable p) {
98          Path rootdir;
99          FileSystem fs;
100         try {
101           rootdir = FSUtils.getRootDir(conf);
102           fs = rootdir.getFileSystem(conf);
103         } catch (IOException e) {
104           LOG.warn("could not find root dir or fs", e);
105           return Status.RESIGNED;
106         }
107         // TODO have to correctly figure out when log splitting has been
108         // interrupted or has encountered a transient error and when it has
109         // encountered a bad non-retry-able persistent error.
110         try {
111           String tmpname =
112             ZKSplitLog.getSplitLogDirTmpComponent(serverName, filename);
113           if (HLogSplitter.splitLogFileToTemp(rootdir, tmpname,
114               fs.getFileStatus(new Path(filename)), fs, conf, p) == false) {
115             return Status.PREEMPTED;
116           }
117         } catch (InterruptedIOException iioe) {
118           LOG.warn("log splitting of " + filename + " interrupted, resigning",
119               iioe);
120           return Status.RESIGNED;
121         } catch (IOException e) {
122           Throwable cause = e.getCause();
123           if (cause instanceof InterruptedException) {
124             LOG.warn("log splitting of " + filename + " interrupted, resigning",
125                 e);
126             return Status.RESIGNED;
127           }
128           LOG.warn("log splitting of " + filename + " failed, returning error",
129               e);
130           return Status.ERR;
131         }
132         return Status.DONE;
133       }
134     });
135   }
136 
137   @Override
138   public void run() {
139    try {
140     LOG.info("SplitLogWorker " + this.serverName + " starting");
141     this.watcher.registerListener(this);
142     int res;
143     // wait for master to create the splitLogZnode
144     res = -1;
145     while (res == -1) {
146       try {
147         res = ZKUtil.checkExists(watcher, watcher.splitLogZNode);
148       } catch (KeeperException e) {
149         // ignore
150         LOG.warn("Exception when checking for " + watcher.splitLogZNode +
151             " ... retrying", e);
152       }
153       if (res == -1) {
154         try {
155           LOG.info(watcher.splitLogZNode + " znode does not exist," +
156               " waiting for master to create one");
157           Thread.sleep(1000);
158         } catch (InterruptedException e) {
159           LOG.debug("Interrupted while waiting for " + watcher.splitLogZNode);
160           assert exitWorker == true;
161         }
162       }
163     }
164 
165     taskLoop();
166    } catch (Throwable t) {
167 	   // only a logical error can cause here. Printing it out 
168 	   // to make debugging easier
169 	   LOG.error("unexpected error ", t);
170    } finally {
171 	   LOG.info("SplitLogWorker " + this.serverName + " exiting");
172    }
173   }
174 
175   /**
176    * Wait for tasks to become available at /hbase/splitlog zknode. Grab a task
177    * one at a time. This policy puts an upper-limit on the number of
178    * simultaneous log splitting that could be happening in a cluster.
179    * <p>
180    * Synchronization using {@link #task_ready_signal_seq} ensures that it will
181    * try to grab every task that has been put up
182    */
183   private void taskLoop() {
184     while (true) {
185       int seq_start = taskReadySeq;
186       List<String> paths = getTaskList();
187       if (paths == null) {
188         LOG.warn("Could not get tasks, did someone remove " +
189             this.watcher.splitLogZNode + " ... worker thread exiting.");
190         return;
191       }
192       int offset = (int)(Math.random() * paths.size());
193       for (int i = 0; i < paths.size(); i ++) {
194         int idx = (i + offset) % paths.size();
195         // don't call ZKSplitLog.getNodeName() because that will lead to
196         // double encoding of the path name
197         grabTask(ZKUtil.joinZNode(watcher.splitLogZNode, paths.get(idx)));
198         if (exitWorker == true) {
199           return;
200         }
201       }
202       synchronized (taskReadyLock) {
203         while (seq_start == taskReadySeq) {
204           try {
205             taskReadyLock.wait();
206           } catch (InterruptedException e) {
207             LOG.info("SplitLogWorker interrupted while waiting for task," +
208               " exiting: " + e.toString());
209             assert exitWorker == true;
210             return;
211           }
212         }
213       }
214     }
215   }
216 
217   /**
218    * try to grab a 'lock' on the task zk node to own and execute the task.
219    * <p>
220    * @param path zk node for the task
221    */
222   private void grabTask(String path) {
223     Stat stat = new Stat();
224     long t = -1;
225     byte[] data;
226     synchronized (grabTaskLock) {
227       currentTask = path;
228       workerInGrabTask = true;
229       if (Thread.interrupted()) {
230         return;
231       }
232     }
233     try {
234       try {
235         if ((data = ZKUtil.getDataNoWatch(this.watcher, path, stat)) == null) {
236           tot_wkr_failed_to_grab_task_no_data.incrementAndGet();
237           return;
238         }
239       } catch (KeeperException e) {
240         LOG.warn("Failed to get data for znode " + path, e);
241         tot_wkr_failed_to_grab_task_exception.incrementAndGet();
242         return;
243       }
244       if (TaskState.TASK_UNASSIGNED.equals(data) == false) {
245         tot_wkr_failed_to_grab_task_owned.incrementAndGet();
246         return;
247       }
248 
249       currentVersion = stat.getVersion();
250       if (ownTask(true) == false) {
251         tot_wkr_failed_to_grab_task_lost_race.incrementAndGet();
252         return;
253       }
254 
255       if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
256         endTask(TaskState.TASK_DONE, tot_wkr_task_acquired_rescan);
257         return;
258       }
259       LOG.info("worker " + serverName + " acquired task " + path);
260       tot_wkr_task_acquired.incrementAndGet();
261       getDataSetWatchAsync();
262 
263       t = System.currentTimeMillis();
264       TaskExecutor.Status status;
265 
266       status = executor.exec(ZKSplitLog.getFileName(currentTask),
267           new CancelableProgressable() {
268 
269         @Override
270         public boolean progress() {
271           if (ownTask(false) == false) {
272             LOG.warn("Failed to heartbeat the task" + currentTask);
273             return false;
274           }
275           return true;
276         }
277       });
278       switch (status) {
279         case DONE:
280           endTask(TaskState.TASK_DONE, tot_wkr_task_done);
281           break;
282         case PREEMPTED:
283           tot_wkr_preempt_task.incrementAndGet();
284           LOG.warn("task execution prempted " + path);
285           break;
286         case ERR:
287           if (!exitWorker) {
288             endTask(TaskState.TASK_ERR, tot_wkr_task_err);
289             break;
290           }
291           // if the RS is exiting then there is probably a tons of stuff
292           // that can go wrong. Resign instead of signaling error.
293           //$FALL-THROUGH$
294         case RESIGNED:
295           if (exitWorker) {
296             LOG.info("task execution interrupted because worker is exiting " +
297                 path);
298             endTask(TaskState.TASK_RESIGNED, tot_wkr_task_resigned);
299           } else {
300             tot_wkr_preempt_task.incrementAndGet();
301             LOG.info("task execution interrupted via zk by manager " +
302                 path);
303           }
304           break;
305       }
306     } finally {
307       if (t > 0) {
308         LOG.info("worker " + serverName + " done with task " + path +
309             " in " + (System.currentTimeMillis() - t) + "ms");
310       }
311       synchronized (grabTaskLock) {
312         workerInGrabTask = false;
313         // clear the interrupt from stopTask() otherwise the next task will
314         // suffer
315         Thread.interrupted();
316       }
317     }
318     return;
319   }
320 
321   /**
322    * Try to own the task by transitioning the zk node data from UNASSIGNED to
323    * OWNED.
324    * <p>
325    * This method is also used to periodically heartbeat the task progress by
326    * transitioning the node from OWNED to OWNED.
327    * <p>
328    * @return true if task path is successfully locked
329    */
330   private boolean ownTask(boolean isFirstTime) {
331     try {
332       Stat stat = this.watcher.getRecoverableZooKeeper().setData(currentTask,
333           TaskState.TASK_OWNED.get(serverName), currentVersion);
334       if (stat == null) {
335         LOG.warn("zk.setData() returned null for path " + currentTask);
336         tot_wkr_task_heartbeat_failed.incrementAndGet();
337         return (false);
338       }
339       currentVersion = stat.getVersion();
340       tot_wkr_task_heartbeat.incrementAndGet();
341       return (true);
342     } catch (KeeperException e) {
343       if (!isFirstTime) {
344         if (e.code().equals(KeeperException.Code.NONODE)) {
345           LOG.warn("NONODE failed to assert ownership for " + currentTask, e);
346         } else if (e.code().equals(KeeperException.Code.BADVERSION)) {
347           LOG.warn("BADVERSION failed to assert ownership for " +
348               currentTask, e);
349         } else {
350           LOG.warn("failed to assert ownership for " + currentTask, e);
351         }
352       }
353     } catch (InterruptedException e1) {
354       LOG.warn("Interrupted while trying to assert ownership of " +
355           currentTask + " " + StringUtils.stringifyException(e1));
356       Thread.currentThread().interrupt();
357     }
358     tot_wkr_task_heartbeat_failed.incrementAndGet();
359     return (false);
360   }
361 
362   /**
363    * endTask() can fail and the only way to recover out of it is for the
364    * {@link SplitLogManager} to timeout the task node.
365    * @param ts
366    * @param ctr
367    */
368   private void endTask(ZKSplitLog.TaskState ts, AtomicLong ctr) {
369     String path = currentTask;
370     currentTask = null;
371     try {
372       if (ZKUtil.setData(this.watcher, path, ts.get(serverName),
373           currentVersion)) {
374         LOG.info("successfully transitioned task " + path +
375             " to final state " + ts);
376         ctr.incrementAndGet();
377         return;
378       }
379       LOG.warn("failed to transistion task " + path + " to end state " + ts +
380           " because of version mismatch ");
381     } catch (KeeperException.BadVersionException bve) {
382       LOG.warn("transisition task " + path + " to " + ts +
383           " failed because of version mismatch", bve);
384     } catch (KeeperException.NoNodeException e) {
385       LOG.fatal("logic error - end task " + path + " " + ts +
386           " failed because task doesn't exist", e);
387     } catch (KeeperException e) {
388       LOG.warn("failed to end task, " + path + " " + ts, e);
389     }
390     tot_wkr_final_transistion_failed.incrementAndGet();
391     return;
392   }
393 
394   void getDataSetWatchAsync() {
395     this.watcher.getRecoverableZooKeeper().getZooKeeper().
396       getData(currentTask, this.watcher,
397       new GetDataAsyncCallback(), null);
398     tot_wkr_get_data_queued.incrementAndGet();
399   }
400 
401   void getDataSetWatchSuccess(String path, byte[] data) {
402     synchronized (grabTaskLock) {
403       if (workerInGrabTask) {
404         // currentTask can change but that's ok
405         String taskpath = currentTask;
406         if (taskpath != null && taskpath.equals(path)) {
407           // have to compare data. cannot compare version because then there
408           // will be race with ownTask()
409           // cannot just check whether the node has been transitioned to
410           // UNASSIGNED because by the time this worker sets the data watch
411           // the node might have made two transitions - from owned by this
412           // worker to unassigned to owned by another worker
413           if (! TaskState.TASK_OWNED.equals(data, serverName) &&
414               ! TaskState.TASK_DONE.equals(data, serverName) &&
415               ! TaskState.TASK_ERR.equals(data, serverName) &&
416               ! TaskState.TASK_RESIGNED.equals(data, serverName)) {
417             LOG.info("task " + taskpath + " preempted from " +
418                 serverName + ", current task state and owner=" +
419                 new String(data));
420             stopTask();
421           }
422         }
423       }
424     }
425   }
426 
427   void getDataSetWatchFailure(String path) {
428     synchronized (grabTaskLock) {
429       if (workerInGrabTask) {
430         // currentTask can change but that's ok
431         String taskpath = currentTask;
432         if (taskpath != null && taskpath.equals(path)) {
433           LOG.info("retrying data watch on " + path);
434           tot_wkr_get_data_retry.incrementAndGet();
435           getDataSetWatchAsync();
436         } else {
437           // no point setting a watch on the task which this worker is not
438           // working upon anymore
439         }
440       }
441     }
442   }
443 
444 
445 
446 
447   @Override
448   public void nodeDataChanged(String path) {
449     // there will be a self generated dataChanged event every time ownTask()
450     // heartbeats the task znode by upping its version
451     synchronized (grabTaskLock) {
452       if (workerInGrabTask) {
453         // currentTask can change
454         String taskpath = currentTask;
455         if (taskpath!= null && taskpath.equals(path)) {
456           getDataSetWatchAsync();
457         }
458       }
459     }
460   }
461 
462 
463   private List<String> getTaskList() {
464     for (int i = 0; i < zkretries; i++) {
465       try {
466         return (ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
467             this.watcher.splitLogZNode));
468       } catch (KeeperException e) {
469         LOG.warn("Could not get children of znode " +
470             this.watcher.splitLogZNode, e);
471         try {
472           Thread.sleep(1000);
473         } catch (InterruptedException e1) {
474           LOG.warn("Interrupted while trying to get task list ...", e1);
475           Thread.currentThread().interrupt();
476           return null;
477         }
478       }
479     }
480     LOG.warn("Tried " + zkretries + " times, still couldn't fetch " +
481         "children of " + watcher.splitLogZNode + " giving up");
482     return null;
483   }
484 
485 
486   @Override
487   public void nodeChildrenChanged(String path) {
488     if(path.equals(watcher.splitLogZNode)) {
489       LOG.debug("tasks arrived or departed");
490       synchronized (taskReadyLock) {
491         taskReadySeq++;
492         taskReadyLock.notify();
493       }
494     }
495   }
496 
497   /**
498    * If the worker is doing a task i.e. splitting a log file then stop the task.
499    * It doesn't exit the worker thread.
500    */
501   void stopTask() {
502     LOG.info("Sending interrupt to stop the worker thread");
503     worker.interrupt(); // TODO interrupt often gets swallowed, do what else?
504   }
505 
506 
507   /**
508    * start the SplitLogWorker thread
509    */
510   public void start() {
511     worker = new Thread(null, this, "SplitLogWorker-" + serverName);
512     exitWorker = false;
513     worker.start();
514     return;
515   }
516 
517   /**
518    * stop the SplitLogWorker thread
519    */
520   public void stop() {
521     exitWorker = true;
522     stopTask();
523   }
524 
525   /**
526    * Asynchronous handler for zk get-data-set-watch on node results.
527    */
528   class GetDataAsyncCallback implements AsyncCallback.DataCallback {
529     private final Log LOG = LogFactory.getLog(GetDataAsyncCallback.class);
530 
531     @Override
532     public void processResult(int rc, String path, Object ctx, byte[] data,
533         Stat stat) {
534       tot_wkr_get_data_result.incrementAndGet();
535       if (rc != 0) {
536         LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
537         getDataSetWatchFailure(path);
538         return;
539       }
540       data = watcher.getRecoverableZooKeeper().removeMetaData(data);
541       getDataSetWatchSuccess(path, data);
542       return;
543     }
544   }
545 
546   /**
547    * Objects implementing this interface actually do the task that has been
548    * acquired by a {@link SplitLogWorker}. Since there isn't a water-tight
549    * guarantee that two workers will not be executing the same task therefore it
550    * is better to have workers prepare the task and then have the
551    * {@link SplitLogManager} commit the work in SplitLogManager.TaskFinisher
552    */
553   static public interface TaskExecutor {
554     static public enum Status {
555       DONE(),
556       ERR(),
557       RESIGNED(),
558       PREEMPTED();
559     }
560     public Status exec(String name, CancelableProgressable p);
561   }
562 }