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}