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 java.util.Optional;
026import org.apache.commons.io.IOUtils;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.fs.FSDataInputStream;
029import org.apache.hadoop.fs.FSDataOutputStream;
030import org.apache.hadoop.fs.FileSystem;
031import org.apache.hadoop.fs.Path;
032import org.apache.hadoop.hbase.NotServingRegionException;
033import org.apache.hadoop.hbase.Server;
034import org.apache.hadoop.hbase.client.RetriesExhaustedException;
035import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
036import org.apache.hadoop.hbase.regionserver.SplitLogWorker.TaskExecutor.Status;
037import org.apache.hadoop.hbase.replication.ReplicationPeerImpl;
038import org.apache.hadoop.hbase.replication.ReplicationUtils;
039import org.apache.hadoop.hbase.replication.SyncReplicationState;
040import org.apache.hadoop.hbase.util.CancelableProgressable;
041import org.apache.hadoop.hbase.util.CommonFSUtils;
042import org.apache.hadoop.hbase.util.ExceptionUtil;
043import org.apache.hadoop.hbase.util.FSUtils;
044import org.apache.hadoop.hbase.util.Pair;
045import org.apache.hadoop.hbase.wal.AbstractWALProvider;
046import org.apache.hadoop.hbase.wal.WALFactory;
047import org.apache.hadoop.hbase.wal.WALSplitter;
048import org.apache.yetus.audience.InterfaceAudience;
049import org.slf4j.Logger;
050import org.slf4j.LoggerFactory;
051
052/**
053 * This worker is spawned in every regionserver, including master. The Worker waits for log
054 * splitting tasks to be put up by the {@link org.apache.hadoop.hbase.master.SplitLogManager}
055 * running in the master and races with other workers in other serves to acquire those tasks. The
056 * coordination is done via coordination engine.
057 * <p>
058 * If a worker has successfully moved the task from state UNASSIGNED to OWNED then it owns the task.
059 * It keeps heart beating the manager by periodically moving the task from UNASSIGNED to OWNED
060 * state. On success it moves the task to TASK_DONE. On unrecoverable error it moves task state to
061 * ERR. If it cannot continue but wants the master to retry the task then it moves the task state to
062 * RESIGNED.
063 * <p>
064 * The manager can take a task away from a worker by moving the task from OWNED to UNASSIGNED. In
065 * the absence of a global lock there is a unavoidable race here - a worker might have just finished
066 * its task when it is stripped of its ownership. Here we rely on the idempotency of the log
067 * splitting task for correctness
068 * @deprecated since 2.4.0 and in 3.0.0, to be removed in 4.0.0, replaced by procedure-based
069 *             distributed WAL splitter, see SplitWALRemoteProcedure
070 */
071@Deprecated
072@InterfaceAudience.Private
073public class SplitLogWorker implements Runnable {
074
075  private static final Logger LOG = LoggerFactory.getLogger(SplitLogWorker.class);
076
077  Thread worker;
078  // thread pool which executes recovery work
079  private final SplitLogWorkerCoordination coordination;
080  private final RegionServerServices server;
081
082  public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
083    TaskExecutor splitTaskExecutor) {
084    this.server = server;
085    this.coordination = hserver.getCoordinatedStateManager().getSplitLogWorkerCoordination();
086    coordination.init(server, conf, splitTaskExecutor, this);
087  }
088
089  public SplitLogWorker(Configuration conf, RegionServerServices server,
090    LastSequenceId sequenceIdChecker, WALFactory factory) {
091    this(server, conf, server, (f, p) -> splitLog(f, p, conf, server, sequenceIdChecker, factory));
092  }
093
094  // returns whether we need to continue the split work
095  private static boolean processSyncReplicationWAL(String name, Configuration conf,
096    RegionServerServices server, FileSystem fs, Path walDir) throws IOException {
097    Path walFile = new Path(walDir, name);
098    String filename = walFile.getName();
099    Optional<String> optSyncPeerId =
100      AbstractWALProvider.getSyncReplicationPeerIdFromWALName(filename);
101    if (!optSyncPeerId.isPresent()) {
102      return true;
103    }
104    String peerId = optSyncPeerId.get();
105    ReplicationPeerImpl peer =
106      server.getReplicationSourceService().getReplicationPeers().getPeer(peerId);
107    if (peer == null || !peer.getPeerConfig().isSyncReplication()) {
108      return true;
109    }
110    Pair<SyncReplicationState, SyncReplicationState> stateAndNewState =
111      peer.getSyncReplicationStateAndNewState();
112    if (
113      stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE)
114        && stateAndNewState.getSecond().equals(SyncReplicationState.NONE)
115    ) {
116      // copy the file to remote and overwrite the previous one
117      String remoteWALDir = peer.getPeerConfig().getRemoteWALDir();
118      Path remoteWALDirForPeer = ReplicationUtils.getPeerRemoteWALDir(remoteWALDir, peerId);
119      Path tmpRemoteWAL = new Path(remoteWALDirForPeer, filename + ".tmp");
120      FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
121      try (FSDataInputStream in = fs.open(walFile);
122        FSDataOutputStream out = remoteFs.createNonRecursive(tmpRemoteWAL, true,
123          CommonFSUtils.getDefaultBufferSize(remoteFs),
124          remoteFs.getDefaultReplication(tmpRemoteWAL), remoteFs.getDefaultBlockSize(tmpRemoteWAL),
125          null)) {
126        IOUtils.copy(in, out);
127      }
128      Path toCommitRemoteWAL =
129        new Path(remoteWALDirForPeer, filename + ReplicationUtils.RENAME_WAL_SUFFIX);
130      // Some FileSystem implementations may not support atomic rename so we need to do it in two
131      // phases
132      FSUtils.renameFile(remoteFs, tmpRemoteWAL, toCommitRemoteWAL);
133      FSUtils.renameFile(remoteFs, toCommitRemoteWAL, new Path(remoteWALDirForPeer, filename));
134    } else if (
135      (stateAndNewState.getFirst().equals(SyncReplicationState.ACTIVE)
136        && stateAndNewState.getSecond().equals(SyncReplicationState.STANDBY))
137        || stateAndNewState.getFirst().equals(SyncReplicationState.STANDBY)
138    ) {
139      // check whether we still need to process this file
140      // actually we only write wal file which name is ended with .syncrep in A state, and after
141      // transiting to a state other than A, we will reopen all the regions so the data in the wal
142      // will be flushed so the wal file will be archived soon. But it is still possible that there
143      // is a server crash when we are transiting from A to S, to simplify the logic of the transit
144      // procedure, here we will also check the remote snapshot directory in state S, so that we do
145      // not need wait until all the wal files with .syncrep suffix to be archived before finishing
146      // the procedure.
147      String remoteWALDir = peer.getPeerConfig().getRemoteWALDir();
148      Path remoteSnapshotDirForPeer = ReplicationUtils.getPeerSnapshotWALDir(remoteWALDir, peerId);
149      FileSystem remoteFs = ReplicationUtils.getRemoteWALFileSystem(conf, remoteWALDir);
150      if (remoteFs.exists(new Path(remoteSnapshotDirForPeer, filename))) {
151        // the file has been replayed when the remote cluster was transited from S to DA, the
152        // content will be replicated back to us so give up split it.
153        LOG.warn("Giveup splitting {} since it has been replayed in the remote cluster and "
154          + "the content will be replicated back", filename);
155        return false;
156      }
157    }
158    return true;
159  }
160
161  /** Returns Result either DONE, RESIGNED, or ERR. */
162  static Status splitLog(String filename, CancelableProgressable p, Configuration conf,
163    RegionServerServices server, LastSequenceId sequenceIdChecker, WALFactory factory) {
164    Path walDir;
165    FileSystem fs;
166    try {
167      walDir = CommonFSUtils.getWALRootDir(conf);
168      fs = walDir.getFileSystem(conf);
169    } catch (IOException e) {
170      LOG.warn("Resigning, could not find root dir or fs", e);
171      return Status.RESIGNED;
172    }
173    try {
174      if (!processSyncReplicationWAL(filename, conf, server, fs, walDir)) {
175        return Status.DONE;
176      }
177    } catch (IOException e) {
178      LOG.warn("failed to process sync replication wal {}", filename, e);
179      return Status.RESIGNED;
180    }
181    // TODO have to correctly figure out when log splitting has been
182    // interrupted or has encountered a transient error and when it has
183    // encountered a bad non-retry-able persistent error.
184    try {
185      SplitLogWorkerCoordination splitLogWorkerCoordination =
186        server.getCoordinatedStateManager() == null
187          ? null
188          : server.getCoordinatedStateManager().getSplitLogWorkerCoordination();
189      if (
190        !WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)), fs, conf, p,
191          sequenceIdChecker, splitLogWorkerCoordination, factory, server)
192      ) {
193        return Status.PREEMPTED;
194      }
195    } catch (InterruptedIOException iioe) {
196      LOG.warn("Resigning, interrupted splitting WAL {}", filename, iioe);
197      return Status.RESIGNED;
198    } catch (IOException e) {
199      if (e instanceof FileNotFoundException) {
200        // A wal file may not exist anymore. Nothing can be recovered so move on
201        LOG.warn("Done, WAL {} does not exist anymore", filename, e);
202        return Status.DONE;
203      }
204      Throwable cause = e.getCause();
205      if (
206        e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
207          || cause instanceof ConnectException || cause instanceof SocketTimeoutException)
208      ) {
209        LOG.warn("Resigning, can't connect to target regionserver splitting WAL {}", filename, e);
210        return Status.RESIGNED;
211      } else if (cause instanceof InterruptedException) {
212        LOG.warn("Resigning, interrupted splitting WAL {}", filename, e);
213        return Status.RESIGNED;
214      }
215      LOG.warn("Error splitting WAL {}", filename, e);
216      return Status.ERR;
217    }
218    LOG.debug("Done splitting WAL {}", filename);
219    return Status.DONE;
220  }
221
222  @Override
223  public void run() {
224    try {
225      LOG.info("SplitLogWorker " + server.getServerName() + " starting");
226      coordination.registerListener();
227      // wait for Coordination Engine is ready
228      boolean res = false;
229      while (!res && !coordination.isStop()) {
230        res = coordination.isReady();
231      }
232      if (!coordination.isStop()) {
233        coordination.taskLoop();
234      }
235    } catch (Throwable t) {
236      if (ExceptionUtil.isInterrupt(t)) {
237        LOG.info("SplitLogWorker interrupted. Exiting. "
238          + (coordination.isStop() ? "" : " (ERROR: exitWorker is not set, exiting anyway)"));
239      } else {
240        // only a logical error can cause here. Printing it out
241        // to make debugging easier
242        LOG.error("unexpected error ", t);
243      }
244    } finally {
245      coordination.removeListener();
246      LOG.info("SplitLogWorker " + server.getServerName() + " exiting");
247    }
248  }
249
250  /**
251   * If the worker is doing a task i.e. splitting a log file then stop the task. It doesn't exit the
252   * worker thread.
253   */
254  public void stopTask() {
255    LOG.info("Sending interrupt to stop the worker thread");
256    worker.interrupt(); // TODO interrupt often gets swallowed, do what else?
257  }
258
259  /**
260   * start the SplitLogWorker thread
261   */
262  public void start() {
263    worker = new Thread(null, this, "SplitLogWorker-" + server.getServerName().toShortString());
264    worker.start();
265  }
266
267  /**
268   * stop the SplitLogWorker thread
269   */
270  public void stop() {
271    coordination.stopProcessingTasks();
272    stopTask();
273  }
274
275  /**
276   * Objects implementing this interface actually do the task that has been acquired by a
277   * {@link SplitLogWorker}. Since there isn't a water-tight guarantee that two workers will not be
278   * executing the same task therefore it is better to have workers prepare the task and then have
279   * the {@link org.apache.hadoop.hbase.master.SplitLogManager} commit the work in
280   * SplitLogManager.TaskFinisher
281   */
282  @FunctionalInterface
283  public interface TaskExecutor {
284    enum Status {
285      DONE(),
286      ERR(),
287      RESIGNED(),
288      PREEMPTED()
289    }
290
291    Status exec(String name, CancelableProgressable p);
292  }
293
294  /**
295   * Returns the number of tasks processed by coordination. This method is used by tests only
296   */
297  public int getTaskReadySeq() {
298    return coordination.getTaskReadySeq();
299  }
300}