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