1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.procedure;
19
20 import java.io.IOException;
21 import java.util.concurrent.Callable;
22 import java.util.concurrent.CountDownLatch;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.errorhandling.ForeignException;
27 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
28 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
29 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
30 import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52 abstract public class Subprocedure implements Callable<Void> {
53 private static final Log LOG = LogFactory.getLog(Subprocedure.class);
54
55
56 final private String barrierName;
57
58
59
60
61
62
63 private final CountDownLatch inGlobalBarrier;
64
65 private final CountDownLatch releasedLocalBarrier;
66
67
68
69
70
71 protected final ForeignExceptionDispatcher monitor;
72
73 protected final long wakeFrequency;
74 protected final TimeoutExceptionInjector executionTimeoutTimer;
75 protected final ProcedureMemberRpcs rpcs;
76
77 private volatile boolean complete = false;
78
79
80
81
82
83
84
85
86
87 public Subprocedure(ProcedureMember member, String procName, ForeignExceptionDispatcher monitor,
88 long wakeFrequency, long timeout) {
89
90 assert member != null : "procedure member should be non-null";
91 assert member.getRpcs() != null : "rpc handlers should be non-null";
92 assert procName != null : "procedure name should be non-null";
93 assert monitor != null : "monitor should be non-null";
94
95
96 this.rpcs = member.getRpcs();
97 this.barrierName = procName;
98 this.monitor = monitor;
99
100
101 this.monitor.addListener(new ForeignExceptionListener() {
102 @Override
103 public void receive(ForeignException ee) {
104
105 if (ee.isRemote()) {
106 LOG.debug("Was remote foreign exception, not redispatching error", ee);
107 return;
108 }
109
110
111 try {
112 rpcs.sendMemberAborted(Subprocedure.this, ee);
113 } catch (IOException e) {
114
115 LOG.error("Can't reach controller, not propagating error", e);
116 }
117 }
118 });
119
120 this.wakeFrequency = wakeFrequency;
121 this.inGlobalBarrier = new CountDownLatch(1);
122 this.releasedLocalBarrier = new CountDownLatch(1);
123
124
125 this.executionTimeoutTimer = new TimeoutExceptionInjector(monitor, timeout);
126 }
127
128 public String getName() {
129 return barrierName;
130 }
131
132 public String getMemberName() {
133 return rpcs.getMemberName();
134 }
135
136 private void rethrowException() throws ForeignException {
137 monitor.rethrowException();
138 }
139
140
141
142
143
144
145
146
147
148
149
150 @SuppressWarnings("finally")
151 final public Void call() {
152 LOG.debug("Starting subprocedure '" + barrierName + "' with timeout " +
153 executionTimeoutTimer.getMaxTime() + "ms");
154
155 executionTimeoutTimer.start();
156
157 try {
158
159 rethrowException();
160 LOG.debug("Subprocedure '" + barrierName + "' starting 'acquire' stage");
161 acquireBarrier();
162 LOG.debug("Subprocedure '" + barrierName + "' locally acquired");
163
164
165 rpcs.sendMemberAcquired(this);
166 LOG.debug("Subprocedure '" + barrierName + "' coordinator notified of 'acquire', waiting on" +
167 " 'reached' or 'abort' from coordinator");
168
169
170 waitForReachedGlobalBarrier();
171 rethrowException();
172
173
174
175
176
177
178
179
180 LOG.debug("Subprocedure '" + barrierName + "' received 'reached' from coordinator.");
181 insideBarrier();
182 LOG.debug("Subprocedure '" + barrierName + "' locally completed");
183
184
185 rpcs.sendMemberCompleted(this);
186 LOG.debug("Subprocedure '" + barrierName + "' has notified controller of completion");
187
188
189 rethrowException();
190 } catch (Exception e) {
191 String msg = null;
192 if (e instanceof InterruptedException) {
193 msg = "Procedure '" + barrierName + "' aborting due to interrupt!" +
194 " Likely due to pool shutdown.";
195 Thread.currentThread().interrupt();
196 } else if (e instanceof ForeignException) {
197 msg = "Subprocedure '" + barrierName + "' aborting due to a ForeignException!";
198 } else {
199 msg = "Subprocedure '" + barrierName + "' failed!";
200 }
201 cancel(msg, e);
202
203 LOG.debug("Subprocedure '" + barrierName + "' running cleanup.");
204 cleanup(e);
205 } finally {
206 releasedLocalBarrier.countDown();
207
208
209 executionTimeoutTimer.complete();
210 complete = true;
211 LOG.debug("Subprocedure '" + barrierName + "' completed.");
212 return null;
213 }
214 }
215
216 boolean isComplete() {
217 return complete;
218 }
219
220
221
222
223 ForeignExceptionSnare getErrorCheckable() {
224 return this.monitor;
225 }
226
227
228
229
230
231
232
233
234
235
236
237
238 abstract public void acquireBarrier() throws ForeignException;
239
240
241
242
243
244
245
246
247
248
249
250 abstract public void insideBarrier() throws ForeignException;
251
252
253
254
255
256
257
258 abstract public void cleanup(Exception e);
259
260
261
262
263
264 public void cancel(String msg, Throwable cause) {
265 LOG.error(msg, cause);
266 complete = true;
267 if (cause instanceof ForeignException) {
268 monitor.receive((ForeignException) cause);
269 } else {
270 monitor.receive(new ForeignException(getMemberName(), cause));
271 }
272 }
273
274
275
276
277
278
279 public void receiveReachedGlobalBarrier() {
280 inGlobalBarrier.countDown();
281 }
282
283
284
285
286
287
288
289
290
291
292
293
294
295 void waitForReachedGlobalBarrier() throws ForeignException, InterruptedException {
296 Procedure.waitForLatch(inGlobalBarrier, monitor, wakeFrequency,
297 barrierName + ":remote acquired");
298 }
299
300
301
302
303
304
305 public void waitForLocallyCompleted() throws ForeignException, InterruptedException {
306 Procedure.waitForLatch(releasedLocalBarrier, monitor, wakeFrequency,
307 barrierName + ":completed");
308 }
309
310
311
312
313
314
315 public static class SubprocedureImpl extends Subprocedure {
316
317 public SubprocedureImpl(ProcedureMember member, String opName,
318 ForeignExceptionDispatcher monitor, long wakeFrequency, long timeout) {
319 super(member, opName, monitor, wakeFrequency, timeout);
320 }
321
322 @Override
323 public void acquireBarrier() throws ForeignException {}
324
325 @Override
326 public void insideBarrier() throws ForeignException {}
327
328 @Override
329 public void cleanup(Exception e) {}
330 };
331 }