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