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