001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.coordination;
021
022import java.util.Collections;
023import java.util.List;
024import java.util.concurrent.atomic.AtomicInteger;
025import java.util.concurrent.atomic.LongAdder;
026
027import org.apache.commons.lang3.RandomUtils;
028import org.apache.commons.lang3.mutable.MutableInt;
029import org.apache.hadoop.conf.Configuration;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.hbase.HConstants;
033import org.apache.hadoop.hbase.ServerName;
034import org.apache.hadoop.hbase.SplitLogCounters;
035import org.apache.hadoop.hbase.SplitLogTask;
036import org.apache.hadoop.hbase.exceptions.DeserializationException;
037import org.apache.hadoop.hbase.log.HBaseMarkers;
038import org.apache.hadoop.hbase.regionserver.RegionServerServices;
039import org.apache.hadoop.hbase.regionserver.SplitLogWorker;
040import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor;
041import org.apache.hadoop.hbase.regionserver.handler.WALSplitterHandler;
042import org.apache.hadoop.hbase.util.CancelableProgressable;
043import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
044import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
045import org.apache.hadoop.hbase.zookeeper.ZKListener;
046import org.apache.hadoop.hbase.zookeeper.ZKMetadata;
047import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
048import org.apache.hadoop.hbase.zookeeper.ZKUtil;
049import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
050import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
051import org.apache.hadoop.util.StringUtils;
052import org.apache.yetus.audience.InterfaceAudience;
053import org.apache.zookeeper.AsyncCallback;
054import org.apache.zookeeper.KeeperException;
055import org.apache.zookeeper.data.Stat;
056import org.slf4j.Logger;
057import org.slf4j.LoggerFactory;
058
059/**
060 * ZooKeeper based implementation of {@link SplitLogWorkerCoordination}
061 * It listen for changes in ZooKeeper and
062 *
063 */
064@InterfaceAudience.Private
065public class ZkSplitLogWorkerCoordination extends ZKListener implements
066    SplitLogWorkerCoordination {
067
068  private static final Logger LOG = LoggerFactory.getLogger(ZkSplitLogWorkerCoordination.class);
069
070  private static final int checkInterval = 5000; // 5 seconds
071  private static final int FAILED_TO_OWN_TASK = -1;
072
073  private  SplitLogWorker worker;
074
075  private TaskExecutor splitTaskExecutor;
076
077  private final AtomicInteger taskReadySeq = new AtomicInteger(0);
078  private volatile String currentTask = null;
079  private int currentVersion;
080  private volatile boolean shouldStop = false;
081  private final Object grabTaskLock = new Object();
082  private boolean workerInGrabTask = false;
083  private int reportPeriod;
084  private RegionServerServices server = null;
085  protected final AtomicInteger tasksInProgress = new AtomicInteger(0);
086  private int maxConcurrentTasks = 0;
087
088  private final ServerName serverName;
089
090  public ZkSplitLogWorkerCoordination(ServerName serverName, ZKWatcher watcher) {
091    super(watcher);
092    this.serverName = serverName;
093  }
094
095  /**
096   * Override handler from {@link ZKListener}
097   */
098  @Override
099  public void nodeChildrenChanged(String path) {
100    if (path.equals(watcher.getZNodePaths().splitLogZNode)) {
101      if (LOG.isTraceEnabled()) {
102        LOG.trace("tasks arrived or departed on " + path);
103      }
104      synchronized (taskReadySeq) {
105        this.taskReadySeq.incrementAndGet();
106        taskReadySeq.notify();
107      }
108    }
109  }
110
111  /**
112   * Override handler from {@link ZKListener}
113   */
114  @Override
115  public void nodeDataChanged(String path) {
116    // there will be a self generated dataChanged event every time attemptToOwnTask()
117    // heartbeats the task znode by upping its version
118    synchronized (grabTaskLock) {
119      if (workerInGrabTask) {
120        // currentTask can change
121        String taskpath = currentTask;
122        if (taskpath != null && taskpath.equals(path)) {
123          getDataSetWatchAsync();
124        }
125      }
126    }
127  }
128
129  /**
130   * Override setter from {@link SplitLogWorkerCoordination}
131   */
132  @Override
133  public void init(RegionServerServices server, Configuration conf,
134      TaskExecutor splitExecutor, SplitLogWorker worker) {
135    this.server = server;
136    this.worker = worker;
137    this.splitTaskExecutor = splitExecutor;
138    maxConcurrentTasks = conf.getInt("hbase.regionserver.wal.max.splitters", DEFAULT_MAX_SPLITTERS);
139    reportPeriod =
140        conf.getInt("hbase.splitlog.report.period",
141          conf.getInt(HConstants.HBASE_SPLITLOG_MANAGER_TIMEOUT,
142            ZKSplitLogManagerCoordination.DEFAULT_TIMEOUT) / 3);
143  }
144
145  /* Support functions for ZooKeeper async callback */
146
147  void getDataSetWatchFailure(String path) {
148    synchronized (grabTaskLock) {
149      if (workerInGrabTask) {
150        // currentTask can change but that's ok
151        String taskpath = currentTask;
152        if (taskpath != null && taskpath.equals(path)) {
153          LOG.info("retrying data watch on " + path);
154          SplitLogCounters.tot_wkr_get_data_retry.increment();
155          getDataSetWatchAsync();
156        } else {
157          // no point setting a watch on the task which this worker is not
158          // working upon anymore
159        }
160      }
161    }
162  }
163
164  public void getDataSetWatchAsync() {
165    watcher.getRecoverableZooKeeper().getZooKeeper()
166        .getData(currentTask, watcher, new GetDataAsyncCallback(), null);
167    SplitLogCounters.tot_wkr_get_data_queued.increment();
168  }
169
170  void getDataSetWatchSuccess(String path, byte[] data) {
171    SplitLogTask slt;
172    try {
173      slt = SplitLogTask.parseFrom(data);
174    } catch (DeserializationException e) {
175      LOG.warn("Failed parse", e);
176      return;
177    }
178    synchronized (grabTaskLock) {
179      if (workerInGrabTask) {
180        // currentTask can change but that's ok
181        String taskpath = currentTask;
182        if (taskpath != null && taskpath.equals(path)) {
183          // have to compare data. cannot compare version because then there
184          // will be race with attemptToOwnTask()
185          // cannot just check whether the node has been transitioned to
186          // UNASSIGNED because by the time this worker sets the data watch
187          // the node might have made two transitions - from owned by this
188          // worker to unassigned to owned by another worker
189          if (!slt.isOwned(serverName) && !slt.isDone(serverName) && !slt.isErr(serverName)
190              && !slt.isResigned(serverName)) {
191            LOG.info("task " + taskpath + " preempted from " + serverName
192                + ", current task state and owner=" + slt.toString());
193            worker.stopTask();
194          }
195        }
196      }
197    }
198  }
199
200  /**
201   * try to grab a 'lock' on the task zk node to own and execute the task.
202   * <p>
203   * @param path zk node for the task
204   * @return boolean value when grab a task success return true otherwise false
205   */
206  private boolean grabTask(String path) {
207    Stat stat = new Stat();
208    byte[] data;
209    synchronized (grabTaskLock) {
210      currentTask = path;
211      workerInGrabTask = true;
212      if (Thread.interrupted()) {
213        return false;
214      }
215    }
216    try {
217      try {
218        if ((data = ZKUtil.getDataNoWatch(watcher, path, stat)) == null) {
219          SplitLogCounters.tot_wkr_failed_to_grab_task_no_data.increment();
220          return false;
221        }
222      } catch (KeeperException e) {
223        LOG.warn("Failed to get data for znode " + path, e);
224        SplitLogCounters.tot_wkr_failed_to_grab_task_exception.increment();
225        return false;
226      }
227      SplitLogTask slt;
228      try {
229        slt = SplitLogTask.parseFrom(data);
230      } catch (DeserializationException e) {
231        LOG.warn("Failed parse data for znode " + path, e);
232        SplitLogCounters.tot_wkr_failed_to_grab_task_exception.increment();
233        return false;
234      }
235      if (!slt.isUnassigned()) {
236        SplitLogCounters.tot_wkr_failed_to_grab_task_owned.increment();
237        return false;
238      }
239
240      currentVersion =
241          attemptToOwnTask(true, watcher, server.getServerName(), path, stat.getVersion());
242      if (currentVersion < 0) {
243        SplitLogCounters.tot_wkr_failed_to_grab_task_lost_race.increment();
244        return false;
245      }
246
247      if (ZKSplitLog.isRescanNode(watcher, currentTask)) {
248        ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
249            new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
250        splitTaskDetails.setTaskNode(currentTask);
251        splitTaskDetails.setCurTaskZKVersion(new MutableInt(currentVersion));
252
253        endTask(new SplitLogTask.Done(server.getServerName()),
254          SplitLogCounters.tot_wkr_task_acquired_rescan, splitTaskDetails);
255        return false;
256      }
257
258      LOG.info("worker " + server.getServerName() + " acquired task " + path);
259      SplitLogCounters.tot_wkr_task_acquired.increment();
260      getDataSetWatchAsync();
261
262      submitTask(path, currentVersion, reportPeriod);
263
264      // after a successful submit, sleep a little bit to allow other RSs to grab the rest tasks
265      try {
266        int sleepTime = RandomUtils.nextInt(0, 500) + 500;
267        Thread.sleep(sleepTime);
268      } catch (InterruptedException e) {
269        LOG.warn("Interrupted while yielding for other region servers", e);
270        Thread.currentThread().interrupt();
271      }
272      return true;
273    } finally {
274      synchronized (grabTaskLock) {
275        workerInGrabTask = false;
276        // clear the interrupt from stopTask() otherwise the next task will
277        // suffer
278        Thread.interrupted();
279      }
280    }
281  }
282
283  /**
284   * Submit a log split task to executor service
285   * @param curTask task to submit
286   * @param curTaskZKVersion current version of task
287   */
288  void submitTask(final String curTask, final int curTaskZKVersion, final int reportPeriod) {
289    final MutableInt zkVersion = new MutableInt(curTaskZKVersion);
290
291    CancelableProgressable reporter = new CancelableProgressable() {
292      private long last_report_at = 0;
293
294      @Override
295      public boolean progress() {
296        long t = EnvironmentEdgeManager.currentTime();
297        if ((t - last_report_at) > reportPeriod) {
298          last_report_at = t;
299          int latestZKVersion =
300              attemptToOwnTask(false, watcher, server.getServerName(), curTask,
301                zkVersion.intValue());
302          if (latestZKVersion < 0) {
303            LOG.warn("Failed to heartbeat the task" + curTask);
304            return false;
305          }
306          zkVersion.setValue(latestZKVersion);
307        }
308        return true;
309      }
310    };
311    ZkSplitLogWorkerCoordination.ZkSplitTaskDetails splitTaskDetails =
312        new ZkSplitLogWorkerCoordination.ZkSplitTaskDetails();
313    splitTaskDetails.setTaskNode(curTask);
314    splitTaskDetails.setCurTaskZKVersion(zkVersion);
315
316    WALSplitterHandler hsh =
317        new WALSplitterHandler(server, this, splitTaskDetails, reporter,
318            this.tasksInProgress, splitTaskExecutor);
319    server.getExecutorService().submit(hsh);
320  }
321
322  /**
323   * @return true if more splitters are available, otherwise false.
324   */
325  private boolean areSplittersAvailable() {
326    return maxConcurrentTasks - tasksInProgress.get() > 0;
327  }
328
329  /**
330   * Try to own the task by transitioning the zk node data from UNASSIGNED to OWNED.
331   * <p>
332   * This method is also used to periodically heartbeat the task progress by transitioning the node
333   * from OWNED to OWNED.
334   * <p>
335   * @param isFirstTime shows whther it's the first attempt.
336   * @param zkw zk wathcer
337   * @param server name
338   * @param task to own
339   * @param taskZKVersion version of the task in zk
340   * @return non-negative integer value when task can be owned by current region server otherwise -1
341   */
342  protected static int attemptToOwnTask(boolean isFirstTime, ZKWatcher zkw,
343      ServerName server, String task, int taskZKVersion) {
344    int latestZKVersion = FAILED_TO_OWN_TASK;
345    try {
346      SplitLogTask slt = new SplitLogTask.Owned(server);
347      Stat stat = zkw.getRecoverableZooKeeper().setData(task, slt.toByteArray(), taskZKVersion);
348      if (stat == null) {
349        LOG.warn("zk.setData() returned null for path " + task);
350        SplitLogCounters.tot_wkr_task_heartbeat_failed.increment();
351        return FAILED_TO_OWN_TASK;
352      }
353      latestZKVersion = stat.getVersion();
354      SplitLogCounters.tot_wkr_task_heartbeat.increment();
355      return latestZKVersion;
356    } catch (KeeperException e) {
357      if (!isFirstTime) {
358        if (e.code().equals(KeeperException.Code.NONODE)) {
359          LOG.warn("NONODE failed to assert ownership for " + task, e);
360        } else if (e.code().equals(KeeperException.Code.BADVERSION)) {
361          LOG.warn("BADVERSION failed to assert ownership for " + task, e);
362        } else {
363          LOG.warn("failed to assert ownership for " + task, e);
364        }
365      }
366    } catch (InterruptedException e1) {
367      LOG.warn("Interrupted while trying to assert ownership of " + task + " "
368          + StringUtils.stringifyException(e1));
369      Thread.currentThread().interrupt();
370    }
371    SplitLogCounters.tot_wkr_task_heartbeat_failed.increment();
372    return FAILED_TO_OWN_TASK;
373  }
374
375  /**
376   * Wait for tasks to become available at /hbase/splitlog zknode. Grab a task one at a time. This
377   * policy puts an upper-limit on the number of simultaneous log splitting that could be happening
378   * in a cluster.
379   * <p>
380   * Synchronization using <code>taskReadySeq</code> ensures that it will try to grab every task
381   * that has been put up
382   * @throws InterruptedException
383   */
384  @Override
385  public void taskLoop() throws InterruptedException {
386    while (!shouldStop) {
387      int seq_start = taskReadySeq.get();
388      List<String> paths;
389      paths = getTaskList();
390      if (paths == null) {
391        LOG.warn("Could not get tasks, did someone remove " + watcher.getZNodePaths().splitLogZNode
392            + " ... worker thread exiting.");
393        return;
394      }
395      // shuffle the paths to prevent different split log worker start from the same log file after
396      // meta log (if any)
397      Collections.shuffle(paths);
398      // pick meta wal firstly
399      int offset = 0;
400      for (int i = 0; i < paths.size(); i++) {
401        if (AbstractFSWALProvider.isMetaFile(paths.get(i))) {
402          offset = i;
403          break;
404        }
405      }
406      int numTasks = paths.size();
407      boolean taskGrabbed = false;
408      for (int i = 0; i < numTasks; i++) {
409        while (!shouldStop) {
410          if (this.areSplittersAvailable()) {
411            if (LOG.isTraceEnabled()) {
412              LOG.trace("Current region server " + server.getServerName()
413                + " is ready to take more tasks, will get task list and try grab tasks again.");
414            }
415            int idx = (i + offset) % paths.size();
416            // don't call ZKSplitLog.getNodeName() because that will lead to
417            // double encoding of the path name
418            taskGrabbed |= grabTask(ZNodePaths.joinZNode(
419                watcher.getZNodePaths().splitLogZNode, paths.get(idx)));
420            break;
421          } else {
422            if (LOG.isTraceEnabled()) {
423              LOG.trace("Current region server " + server.getServerName() + " has "
424                + this.tasksInProgress.get() + " tasks in progress and can't take more.");
425            }
426            Thread.sleep(100);
427          }
428        }
429        if (shouldStop) {
430          return;
431        }
432      }
433      if (!taskGrabbed && !shouldStop) {
434        // do not grab any tasks, sleep a little bit to reduce zk request.
435        Thread.sleep(1000);
436      }
437      SplitLogCounters.tot_wkr_task_grabing.increment();
438      synchronized (taskReadySeq) {
439        while (seq_start == taskReadySeq.get()) {
440          taskReadySeq.wait(checkInterval);
441        }
442      }
443    }
444  }
445
446  private List<String> getTaskList() throws InterruptedException {
447    List<String> childrenPaths = null;
448    long sleepTime = 1000;
449    // It will be in loop till it gets the list of children or
450    // it will come out if worker thread exited.
451    while (!shouldStop) {
452      try {
453        childrenPaths = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
454          watcher.getZNodePaths().splitLogZNode);
455        if (childrenPaths != null) {
456          return childrenPaths;
457        }
458      } catch (KeeperException e) {
459        LOG.warn("Could not get children of znode " + watcher.getZNodePaths().splitLogZNode, e);
460      }
461      LOG.debug("Retry listChildren of znode " + watcher.getZNodePaths().splitLogZNode
462          + " after sleep for " + sleepTime + "ms!");
463      Thread.sleep(sleepTime);
464    }
465    return childrenPaths;
466  }
467
468  @Override
469  public void markCorrupted(Path rootDir, String name, FileSystem fs) {
470    ZKSplitLog.markCorrupted(rootDir, name, fs);
471  }
472
473  @Override
474  public boolean isReady() throws InterruptedException {
475    int result = -1;
476    try {
477      result = ZKUtil.checkExists(watcher, watcher.getZNodePaths().splitLogZNode);
478    } catch (KeeperException e) {
479      // ignore
480      LOG.warn("Exception when checking for " + watcher.getZNodePaths().splitLogZNode
481          + " ... retrying", e);
482    }
483    if (result == -1) {
484      LOG.info(watcher.getZNodePaths().splitLogZNode
485          + " znode does not exist, waiting for master to create");
486      Thread.sleep(1000);
487    }
488    return (result != -1);
489  }
490
491  @Override
492  public int getTaskReadySeq() {
493    return taskReadySeq.get();
494  }
495
496  @Override
497  public void registerListener() {
498    watcher.registerListener(this);
499  }
500
501  @Override
502  public void removeListener() {
503    watcher.unregisterListener(this);
504  }
505
506
507  @Override
508  public void stopProcessingTasks() {
509    this.shouldStop = true;
510
511  }
512
513  @Override
514  public boolean isStop() {
515    return shouldStop;
516  }
517
518  /**
519   * Asynchronous handler for zk get-data-set-watch on node results.
520   */
521  class GetDataAsyncCallback implements AsyncCallback.DataCallback {
522    private final Logger LOG = LoggerFactory.getLogger(GetDataAsyncCallback.class);
523
524    @Override
525    public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
526      SplitLogCounters.tot_wkr_get_data_result.increment();
527      if (rc != 0) {
528        LOG.warn("getdata rc = " + KeeperException.Code.get(rc) + " " + path);
529        getDataSetWatchFailure(path);
530        return;
531      }
532      data = ZKMetadata.removeMetaData(data);
533      getDataSetWatchSuccess(path, data);
534    }
535  }
536
537  /*
538   * Next part is related to WALSplitterHandler
539   */
540  /**
541   * endTask() can fail and the only way to recover out of it is for the
542   * {@link org.apache.hadoop.hbase.master.SplitLogManager} to timeout the task node.
543   * @param slt
544   * @param ctr
545   */
546  @Override
547  public void endTask(SplitLogTask slt, LongAdder ctr, SplitTaskDetails details) {
548    ZkSplitTaskDetails zkDetails = (ZkSplitTaskDetails) details;
549    String task = zkDetails.getTaskNode();
550    int taskZKVersion = zkDetails.getCurTaskZKVersion().intValue();
551    try {
552      if (ZKUtil.setData(watcher, task, slt.toByteArray(), taskZKVersion)) {
553        LOG.info("successfully transitioned task " + task + " to final state " + slt);
554        if (ctr != null) {
555          ctr.increment();
556        }
557        return;
558      }
559      LOG.warn("failed to transistion task " + task + " to end state " + slt
560          + " because of version mismatch ");
561    } catch (KeeperException.BadVersionException bve) {
562      LOG.warn("transisition task " + task + " to " + slt + " failed because of version mismatch",
563        bve);
564    } catch (KeeperException.NoNodeException e) {
565      LOG.error(HBaseMarkers.FATAL,
566        "logic error - end task " + task + " " + slt + " failed because task doesn't exist", e);
567    } catch (KeeperException e) {
568      LOG.warn("failed to end task, " + task + " " + slt, e);
569    }
570    SplitLogCounters.tot_wkr_final_transition_failed.increment();
571  }
572
573  /**
574   * When ZK-based implementation wants to complete the task, it needs to know task znode and
575   * current znode cversion (needed for subsequent update operation).
576   */
577  public static class ZkSplitTaskDetails implements SplitTaskDetails {
578    private String taskNode;
579    private MutableInt curTaskZKVersion;
580
581    public ZkSplitTaskDetails() {
582    }
583
584    public ZkSplitTaskDetails(String taskNode, MutableInt curTaskZKVersion) {
585      this.taskNode = taskNode;
586      this.curTaskZKVersion = curTaskZKVersion;
587    }
588
589    public String getTaskNode() {
590      return taskNode;
591    }
592
593    public void setTaskNode(String taskNode) {
594      this.taskNode = taskNode;
595    }
596
597    public MutableInt getCurTaskZKVersion() {
598      return curTaskZKVersion;
599    }
600
601    public void setCurTaskZKVersion(MutableInt curTaskZKVersion) {
602      this.curTaskZKVersion = curTaskZKVersion;
603    }
604
605    @Override
606    public String getWALFile() {
607      return ZKSplitLog.getFileName(taskNode);
608    }
609  }
610
611}