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}