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