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.ArrayList;
22 import java.util.List;
23 import java.util.concurrent.Callable;
24 import java.util.concurrent.CountDownLatch;
25 import java.util.concurrent.TimeUnit;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.classification.InterfaceAudience;
30 import org.apache.hadoop.classification.InterfaceStability;
31 import org.apache.hadoop.hbase.errorhandling.ForeignException;
32 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
33 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionListener;
34 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
35 import org.apache.hadoop.hbase.errorhandling.TimeoutExceptionInjector;
36
37 import com.google.common.collect.Lists;
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68 @InterfaceAudience.Public
69 @InterfaceStability.Evolving
70 public class Procedure implements Callable<Void>, ForeignExceptionListener {
71 private static final Log LOG = LogFactory.getLog(Procedure.class);
72
73
74
75
76
77
78 final private String procName;
79
80 final private byte[] args;
81
82
83
84
85
86 final CountDownLatch acquiredBarrierLatch;
87
88 final CountDownLatch releasedBarrierLatch;
89
90 final CountDownLatch completedLatch;
91
92 private final ForeignExceptionDispatcher monitor;
93
94
95
96
97
98
99 protected final long wakeFrequency;
100 protected final TimeoutExceptionInjector timeoutInjector;
101
102
103
104
105
106
107 private Object joinBarrierLock = new Object();
108 private final List<String> acquiringMembers;
109 private final List<String> inBarrierMembers;
110 private ProcedureCoordinator coord;
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125 public Procedure(ProcedureCoordinator coord, ForeignExceptionDispatcher monitor, long wakeFreq,
126 long timeout, String procName, byte[] args, List<String> expectedMembers) {
127 this.coord = coord;
128 this.acquiringMembers = new ArrayList<String>(expectedMembers);
129 this.inBarrierMembers = new ArrayList<String>(acquiringMembers.size());
130 this.procName = procName;
131 this.args = args;
132 this.monitor = monitor;
133 this.wakeFrequency = wakeFreq;
134
135 int count = expectedMembers.size();
136 this.acquiredBarrierLatch = new CountDownLatch(count);
137 this.releasedBarrierLatch = new CountDownLatch(count);
138 this.completedLatch = new CountDownLatch(1);
139 this.timeoutInjector = new TimeoutExceptionInjector(monitor, timeout);
140 }
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157 public Procedure(ProcedureCoordinator coord, long wakeFreq, long timeout,
158 String procName, byte[] args, List<String> expectedMembers) {
159 this(coord, new ForeignExceptionDispatcher(), wakeFreq, timeout, procName, args,
160 expectedMembers);
161 }
162
163 public String getName() {
164 return procName;
165 }
166
167
168
169
170 public String getStatus() {
171 String waiting, done;
172 synchronized (joinBarrierLock) {
173 waiting = acquiringMembers.toString();
174 done = inBarrierMembers.toString();
175 }
176 return "Procedure " + procName + " { waiting=" + waiting + " done="+ done + " }";
177 }
178
179
180
181
182
183 public ForeignExceptionDispatcher getErrorMonitor() {
184 return monitor;
185 }
186
187
188
189
190
191
192 @Override
193 @SuppressWarnings("finally")
194 final public Void call() {
195 LOG.info("Starting procedure '" + procName + "'");
196
197 timeoutInjector.start();
198
199
200 try {
201
202 monitor.rethrowException();
203 LOG.debug("Procedure '" + procName + "' starting 'acquire'");
204 sendGlobalBarrierStart();
205
206
207 LOG.debug("Waiting for all members to 'acquire'");
208 waitForLatch(acquiredBarrierLatch, monitor, wakeFrequency, "acquired");
209 monitor.rethrowException();
210
211 LOG.debug("Procedure '" + procName + "' starting 'in-barrier' execution.");
212 sendGlobalBarrierReached();
213
214
215 waitForLatch(releasedBarrierLatch, monitor, wakeFrequency, "released");
216
217
218 monitor.rethrowException();
219 LOG.info("Procedure '" + procName + "' execution completed");
220 } catch (Exception e) {
221 if (e instanceof InterruptedException) {
222 Thread.currentThread().interrupt();
223 }
224 String msg = "Procedure '" + procName +"' execution failed!";
225 LOG.error(msg, e);
226 receive(new ForeignException(getName(), e));
227 } finally {
228 LOG.debug("Running finish phase.");
229 sendGlobalBarrierComplete();
230 completedLatch.countDown();
231
232
233 timeoutInjector.complete();
234 return null;
235 }
236 }
237
238
239
240
241
242
243 public void sendGlobalBarrierStart() throws ForeignException {
244
245 LOG.debug("Starting procedure '" + procName + "', kicking off acquire phase on members.");
246 try {
247
248
249 coord.getRpcs().sendGlobalBarrierAcquire(this, args, Lists.newArrayList(this.acquiringMembers));
250 } catch (IOException e) {
251 coord.rpcConnectionFailure("Can't reach controller.", e);
252 } catch (IllegalArgumentException e) {
253 throw new ForeignException(getName(), e);
254 }
255 }
256
257
258
259
260
261
262
263
264 public void sendGlobalBarrierReached() throws ForeignException {
265 try {
266
267 coord.getRpcs().sendGlobalBarrierReached(this, Lists.newArrayList(inBarrierMembers));
268 } catch (IOException e) {
269 coord.rpcConnectionFailure("Can't reach controller.", e);
270 }
271 }
272
273
274
275
276
277
278 public void sendGlobalBarrierComplete() {
279 LOG.debug("Finished coordinator procedure - removing self from list of running procedures");
280 try {
281 coord.getRpcs().resetMembers(this);
282 } catch (IOException e) {
283 coord.rpcConnectionFailure("Failed to reset procedure:" + procName, e);
284 }
285 }
286
287
288
289
290
291
292
293
294
295 public void barrierAcquiredByMember(String member) {
296 LOG.debug("member: '" + member + "' joining acquired barrier for procedure '" + procName
297 + "' on coordinator");
298 if (this.acquiringMembers.contains(member)) {
299 synchronized (joinBarrierLock) {
300 if (this.acquiringMembers.remove(member)) {
301 this.inBarrierMembers.add(member);
302 acquiredBarrierLatch.countDown();
303 }
304 }
305 LOG.debug("Waiting on: " + acquiredBarrierLatch + " remaining members to acquire global barrier");
306 } else {
307 LOG.warn("Member " + member + " joined barrier, but we weren't waiting on it to join." +
308 " Continuing on.");
309 }
310 }
311
312
313
314
315
316
317 public void barrierReleasedByMember(String member) {
318 boolean removed = false;
319 synchronized (joinBarrierLock) {
320 removed = this.inBarrierMembers.remove(member);
321 if (removed) {
322 releasedBarrierLatch.countDown();
323 }
324 }
325 if (removed) {
326 LOG.debug("Member: '" + member + "' released barrier for procedure'" + procName
327 + "', counting down latch. Waiting for " + releasedBarrierLatch.getCount()
328 + " more");
329 } else {
330 LOG.warn("Member: '" + member + "' released barrier for procedure'" + procName
331 + "', but we weren't waiting on it to release!");
332 }
333 }
334
335
336
337
338
339
340
341
342 public void waitForCompleted() throws ForeignException, InterruptedException {
343 waitForLatch(completedLatch, monitor, wakeFrequency, procName + " completed");
344 }
345
346
347
348
349 @Override
350 public void receive(ForeignException e) {
351 monitor.receive(e);
352 }
353
354
355
356
357
358
359
360
361
362
363
364
365 public static void waitForLatch(CountDownLatch latch, ForeignExceptionSnare monitor,
366 long wakeFrequency, String latchDescription) throws ForeignException,
367 InterruptedException {
368 boolean released = false;
369 while (!released) {
370 if (monitor != null) {
371 monitor.rethrowException();
372 }
373
374
375
376 released = latch.await(wakeFrequency, TimeUnit.MILLISECONDS);
377 }
378 }
379 }