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 rethrowException();
164
165
166 rpcs.sendMemberAcquired(this);
167 LOG.debug("Subprocedure '" + barrierName + "' coordinator notified of 'acquire', waiting on" +
168 " 'reached' or 'abort' from coordinator");
169
170
171 waitForReachedGlobalBarrier();
172 rethrowException();
173
174
175
176
177
178
179
180
181 LOG.debug("Subprocedure '" + barrierName + "' received 'reached' from coordinator.");
182 byte[] dataToCoordinator = insideBarrier();
183 LOG.debug("Subprocedure '" + barrierName + "' locally completed");
184 rethrowException();
185
186
187 rpcs.sendMemberCompleted(this, dataToCoordinator);
188 LOG.debug("Subprocedure '" + barrierName + "' has notified controller of completion");
189
190
191 rethrowException();
192 } catch (Exception e) {
193 String msg = null;
194 if (e instanceof InterruptedException) {
195 msg = "Procedure '" + barrierName + "' aborting due to interrupt!" +
196 " Likely due to pool shutdown.";
197 Thread.currentThread().interrupt();
198 } else if (e instanceof ForeignException) {
199 msg = "Subprocedure '" + barrierName + "' aborting due to a ForeignException!";
200 } else {
201 msg = "Subprocedure '" + barrierName + "' failed!";
202 }
203 cancel(msg, e);
204
205 LOG.debug("Subprocedure '" + barrierName + "' running cleanup.");
206 cleanup(e);
207 } finally {
208 releasedLocalBarrier.countDown();
209
210
211 executionTimeoutTimer.complete();
212 complete = true;
213 LOG.debug("Subprocedure '" + barrierName + "' completed.");
214 return null;
215 }
216 }
217
218 boolean isComplete() {
219 return complete;
220 }
221
222
223
224
225 ForeignExceptionSnare getErrorCheckable() {
226 return this.monitor;
227 }
228
229
230
231
232
233
234
235
236
237
238
239
240 abstract public void acquireBarrier() throws ForeignException;
241
242
243
244
245
246
247
248
249
250
251
252
253 abstract public byte[] insideBarrier() throws ForeignException;
254
255
256
257
258
259
260
261 abstract public void cleanup(Exception e);
262
263
264
265
266
267 public void cancel(String msg, Throwable cause) {
268 LOG.error(msg, cause);
269 complete = true;
270 if (cause instanceof ForeignException) {
271 monitor.receive((ForeignException) cause);
272 } else {
273 monitor.receive(new ForeignException(getMemberName(), cause));
274 }
275 }
276
277
278
279
280
281
282 public void receiveReachedGlobalBarrier() {
283 inGlobalBarrier.countDown();
284 }
285
286
287
288
289
290
291
292
293
294
295
296
297
298 void waitForReachedGlobalBarrier() throws ForeignException, InterruptedException {
299 Procedure.waitForLatch(inGlobalBarrier, monitor, wakeFrequency,
300 barrierName + ":remote acquired");
301 }
302
303
304
305
306
307
308 public void waitForLocallyCompleted() throws ForeignException, InterruptedException {
309 Procedure.waitForLatch(releasedLocalBarrier, monitor, wakeFrequency,
310 barrierName + ":completed");
311 }
312
313
314
315
316
317
318 public static class SubprocedureImpl extends Subprocedure {
319
320 public SubprocedureImpl(ProcedureMember member, String opName,
321 ForeignExceptionDispatcher monitor, long wakeFrequency, long timeout) {
322 super(member, opName, monitor, wakeFrequency, timeout);
323 }
324
325 @Override
326 public void acquireBarrier() throws ForeignException {}
327
328 @Override
329 public byte[] insideBarrier() throws ForeignException {
330 return new byte[0];
331 }
332
333 @Override
334 public void cleanup(Exception e) {}
335 };
336 }