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 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.io.FileNotFoundException;
022import java.io.IOException;
023import java.io.InterruptedIOException;
024import java.net.ConnectException;
025import java.net.SocketTimeoutException;
026
027import org.apache.yetus.audience.InterfaceAudience;
028import org.slf4j.Logger;
029import org.slf4j.LoggerFactory;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.fs.FileSystem;
032import org.apache.hadoop.fs.Path;
033import org.apache.hadoop.hbase.NotServingRegionException;
034import org.apache.hadoop.hbase.Server;
035import org.apache.hadoop.hbase.client.RetriesExhaustedException;
036import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
037import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
038import org.apache.hadoop.hbase.wal.WALFactory;
039import org.apache.hadoop.hbase.wal.WALSplitter;
040import org.apache.hadoop.hbase.util.CancelableProgressable;
041import org.apache.hadoop.hbase.util.ExceptionUtil;
042import org.apache.hadoop.hbase.util.FSUtils;
043
044import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
045
046/**
047 * This worker is spawned in every regionserver, including master. The Worker waits for log
048 * splitting tasks to be put up by the {@link org.apache.hadoop.hbase.master.SplitLogManager}
049 * running in the master and races with other workers in other serves to acquire those tasks.
050 * The coordination is done via coordination engine.
051 * <p>
052 * If a worker has successfully moved the task from state UNASSIGNED to OWNED then it owns the task.
053 * It keeps heart beating the manager by periodically moving the task from UNASSIGNED to OWNED
054 * state. On success it moves the task to TASK_DONE. On unrecoverable error it moves task state to
055 * ERR. If it cannot continue but wants the master to retry the task then it moves the task state to
056 * RESIGNED.
057 * <p>
058 * The manager can take a task away from a worker by moving the task from OWNED to UNASSIGNED. In
059 * the absence of a global lock there is a unavoidable race here - a worker might have just finished
060 * its task when it is stripped of its ownership. Here we rely on the idempotency of the log
061 * splitting task for correctness
062 */
063@InterfaceAudience.Private
064public class SplitLogWorker implements Runnable {
065
066  private static final Logger LOG = LoggerFactory.getLogger(SplitLogWorker.class);
067
068  Thread worker;
069  // thread pool which executes recovery work
070  private SplitLogWorkerCoordination coordination;
071  private Configuration conf;
072  private RegionServerServices server;
073
074  public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
075      TaskExecutor splitTaskExecutor) {
076    this.server = server;
077    this.conf = conf;
078    this.coordination = hserver.getCoordinatedStateManager().getSplitLogWorkerCoordination();
079    coordination.init(server, conf, splitTaskExecutor, this);
080  }
081
082  public SplitLogWorker(Configuration conf, RegionServerServices server,
083      LastSequenceId sequenceIdChecker, WALFactory factory) {
084    this(server, conf, server, (f, p) -> splitLog(f, p, conf, server, sequenceIdChecker, factory));
085  }
086
087  static Status splitLog(String filename, CancelableProgressable p, Configuration conf,
088      RegionServerServices server, LastSequenceId sequenceIdChecker, WALFactory factory) {
089    Path walDir;
090    FileSystem fs;
091    try {
092      walDir = FSUtils.getWALRootDir(conf);
093      fs = walDir.getFileSystem(conf);
094    } catch (IOException e) {
095      LOG.warn("could not find root dir or fs", e);
096      return Status.RESIGNED;
097    }
098    // TODO have to correctly figure out when log splitting has been
099    // interrupted or has encountered a transient error and when it has
100    // encountered a bad non-retry-able persistent error.
101    try {
102      SplitLogWorkerCoordination splitLogWorkerCoordination =
103          server.getCoordinatedStateManager() == null ? null
104              : server.getCoordinatedStateManager().getSplitLogWorkerCoordination();
105      if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)), fs, conf,
106        p, sequenceIdChecker, splitLogWorkerCoordination, factory)) {
107        return Status.PREEMPTED;
108      }
109    } catch (InterruptedIOException iioe) {
110      LOG.warn("log splitting of " + filename + " interrupted, resigning", iioe);
111      return Status.RESIGNED;
112    } catch (IOException e) {
113      if (e instanceof FileNotFoundException) {
114        // A wal file may not exist anymore. Nothing can be recovered so move on
115        LOG.warn("WAL {} does not exist anymore", filename, e);
116        return Status.DONE;
117      }
118      Throwable cause = e.getCause();
119      if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
120          || cause instanceof ConnectException || cause instanceof SocketTimeoutException)) {
121        LOG.warn("log replaying of " + filename + " can't connect to the target regionserver, "
122            + "resigning",
123          e);
124        return Status.RESIGNED;
125      } else if (cause instanceof InterruptedException) {
126        LOG.warn("log splitting of " + filename + " interrupted, resigning", e);
127        return Status.RESIGNED;
128      }
129      LOG.warn("log splitting of " + filename + " failed, returning error", e);
130      return Status.ERR;
131    }
132    return Status.DONE;
133  }
134
135
136  @Override
137  public void run() {
138    try {
139      LOG.info("SplitLogWorker " + server.getServerName() + " starting");
140      coordination.registerListener();
141      // wait for Coordination Engine is ready
142      boolean res = false;
143      while (!res && !coordination.isStop()) {
144        res = coordination.isReady();
145      }
146      if (!coordination.isStop()) {
147        coordination.taskLoop();
148      }
149    } catch (Throwable t) {
150      if (ExceptionUtil.isInterrupt(t)) {
151        LOG.info("SplitLogWorker interrupted. Exiting. " + (coordination.isStop() ? "" :
152            " (ERROR: exitWorker is not set, exiting anyway)"));
153      } else {
154        // only a logical error can cause here. Printing it out
155        // to make debugging easier
156        LOG.error("unexpected error ", t);
157      }
158    } finally {
159      coordination.removeListener();
160      LOG.info("SplitLogWorker " + server.getServerName() + " exiting");
161    }
162  }
163
164  /**
165   * If the worker is doing a task i.e. splitting a log file then stop the task.
166   * It doesn't exit the worker thread.
167   */
168  public void stopTask() {
169    LOG.info("Sending interrupt to stop the worker thread");
170    worker.interrupt(); // TODO interrupt often gets swallowed, do what else?
171  }
172
173  /**
174   * start the SplitLogWorker thread
175   */
176  public void start() {
177    worker = new Thread(null, this, "SplitLogWorker-" + server.getServerName().toShortString());
178    worker.start();
179  }
180
181  /**
182   * stop the SplitLogWorker thread
183   */
184  public void stop() {
185    coordination.stopProcessingTasks();
186    stopTask();
187  }
188
189  /**
190   * Objects implementing this interface actually do the task that has been
191   * acquired by a {@link SplitLogWorker}. Since there isn't a water-tight
192   * guarantee that two workers will not be executing the same task therefore it
193   * is better to have workers prepare the task and then have the
194   * {@link org.apache.hadoop.hbase.master.SplitLogManager} commit the work in
195   * SplitLogManager.TaskFinisher
196   */
197  public interface TaskExecutor {
198    enum Status {
199      DONE(),
200      ERR(),
201      RESIGNED(),
202      PREEMPTED()
203    }
204    Status exec(String name, CancelableProgressable p);
205  }
206
207  /**
208   * Returns the number of tasks processed by coordination.
209   * This method is used by tests only
210   */
211  @VisibleForTesting
212  public int getTaskReadySeq() {
213    return coordination.getTaskReadySeq();
214  }
215}