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