1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.regionserver;
20
21 import java.io.FileNotFoundException;
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.net.ConnectException;
25 import java.net.SocketTimeoutException;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.FileSystem;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hbase.NotServingRegionException;
34 import org.apache.hadoop.hbase.Server;
35 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
36 import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
37 import org.apache.hadoop.hbase.coordination.SplitLogWorkerCoordination;
38 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
39 import org.apache.hadoop.hbase.wal.WALFactory;
40 import org.apache.hadoop.hbase.wal.WALSplitter;
41 import org.apache.hadoop.hbase.util.CancelableProgressable;
42 import org.apache.hadoop.hbase.util.ExceptionUtil;
43 import org.apache.hadoop.hbase.util.FSUtils;
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62 @InterfaceAudience.Private
63 public class SplitLogWorker implements Runnable {
64
65 private static final Log LOG = LogFactory.getLog(SplitLogWorker.class);
66
67 Thread worker;
68
69 private SplitLogWorkerCoordination coordination;
70 private Configuration conf;
71 private RegionServerServices server;
72
73 public SplitLogWorker(Server hserver, Configuration conf, RegionServerServices server,
74 TaskExecutor splitTaskExecutor) {
75 this.server = server;
76 this.conf = conf;
77 this.coordination =
78 ((BaseCoordinatedStateManager) hserver.getCoordinatedStateManager())
79 .getSplitLogWorkerCoordination();
80 this.server = server;
81 coordination.init(server, conf, splitTaskExecutor, this);
82 }
83
84 public SplitLogWorker(final Server hserver, final Configuration conf,
85 final RegionServerServices rsServices, final LastSequenceId sequenceIdChecker,
86 final WALFactory factory) {
87 this(hserver, conf, rsServices, new TaskExecutor() {
88 @Override
89 public Status exec(String filename, RecoveryMode mode, CancelableProgressable p) {
90 Path walDir;
91 FileSystem fs;
92 try {
93 walDir = FSUtils.getWALRootDir(conf);
94 fs = walDir.getFileSystem(conf);
95 } catch (IOException e) {
96 LOG.warn("could not find root dir or fs", e);
97 return Status.RESIGNED;
98 }
99
100
101
102 try {
103 if (!WALSplitter.splitLogFile(walDir, fs.getFileStatus(new Path(walDir, filename)),
104 fs, conf, p, sequenceIdChecker, rsServices.getCoordinatedStateManager(), mode, factory)) {
105 return Status.PREEMPTED;
106 }
107 } catch (InterruptedIOException iioe) {
108 LOG.warn("log splitting of " + filename + " interrupted, resigning", iioe);
109 return Status.RESIGNED;
110 } catch (IOException e) {
111 if (e instanceof FileNotFoundException) {
112
113 LOG.warn("WAL " + filename + " does not exist anymore", e);
114 return Status.DONE;
115 }
116 Throwable cause = e.getCause();
117 if (e instanceof RetriesExhaustedException && (cause instanceof NotServingRegionException
118 || cause instanceof ConnectException
119 || cause instanceof SocketTimeoutException)) {
120 LOG.warn("log replaying of " + filename + " can't connect to the target regionserver, "
121 + "resigning", e);
122 return Status.RESIGNED;
123 } else if (cause instanceof InterruptedException) {
124 LOG.warn("log splitting of " + filename + " interrupted, resigning", e);
125 return Status.RESIGNED;
126 }
127 LOG.warn("log splitting of " + filename + " failed, returning error", e);
128 return Status.ERR;
129 }
130 return Status.DONE;
131 }
132 });
133 }
134
135 @Override
136 public void run() {
137 try {
138 LOG.info("SplitLogWorker " + server.getServerName() + " starting");
139 coordination.registerListener();
140
141 boolean res = false;
142 while (!res && !coordination.isStop()) {
143 res = coordination.isReady();
144 }
145 if (!coordination.isStop()) {
146 coordination.taskLoop();
147 }
148 } catch (Throwable t) {
149 if (ExceptionUtil.isInterrupt(t)) {
150 LOG.info("SplitLogWorker interrupted. Exiting. " + (coordination.isStop() ? "" :
151 " (ERROR: exitWorker is not set, exiting anyway)"));
152 } else {
153
154
155 LOG.error("unexpected error ", t);
156 }
157 } finally {
158 coordination.removeListener();
159 LOG.info("SplitLogWorker " + server.getServerName() + " exiting");
160 }
161 }
162
163
164
165
166
167 public void stopTask() {
168 LOG.info("Sending interrupt to stop the worker thread");
169 worker.interrupt();
170 }
171
172
173
174
175 public void start() {
176 worker = new Thread(null, this, "SplitLogWorker-" + server.getServerName().toShortString());
177 worker.start();
178 }
179
180
181
182
183 public void stop() {
184 coordination.stopProcessingTasks();
185 stopTask();
186 }
187
188
189
190
191
192
193
194
195
196 public interface TaskExecutor {
197 enum Status {
198 DONE(),
199 ERR(),
200 RESIGNED(),
201 PREEMPTED()
202 }
203 Status exec(String name, RecoveryMode mode, CancelableProgressable p);
204 }
205
206
207
208
209
210 public int getTaskReadySeq() {
211 return coordination.getTaskReadySeq();
212 }
213 }