/
AsyncDispatcher.java
423 lines (374 loc) · 15 KB
/
AsyncDispatcher.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
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
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.tez.common;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.service.CompositeService;
import org.apache.hadoop.util.ShutdownHookManager;
import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.event.Event;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.tez.dag.api.TezConfiguration;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import com.google.common.collect.Maps;
/**
* Dispatches {@link Event}s in a separate thread. Currently only single thread
* does that. Potentially there could be multiple channels for each event type
* class and a thread pool can be used to dispatch the events.
*/
@SuppressWarnings({"rawtypes", "unchecked"})
@Private
public class AsyncDispatcher extends CompositeService implements Dispatcher {
private static final Logger LOG = LoggerFactory.getLogger(AsyncDispatcher.class);
private final String name;
private final BlockingQueue<Event> eventQueue;
private volatile boolean stopped = false;
// Configuration flag for enabling/disabling draining dispatcher's events on
// stop functionality.
private volatile boolean drainEventsOnStop = false;
// Indicates all the remaining dispatcher's events on stop have been drained
// and processed.
private volatile boolean drained = true;
private Object waitForDrained = new Object();
// For drainEventsOnStop enabled only, block newly coming events into the
// queue while stopping.
private volatile boolean blockNewEvents = false;
private EventHandler handlerInstance = new GenericEventHandler();
private Thread eventHandlingThread;
protected final Map<Class<? extends Enum>, EventHandler> eventHandlers = Maps.newHashMap();
protected final Map<Class<? extends Enum>, AsyncDispatcher> eventDispatchers = Maps.newHashMap();
protected final Map<Class<? extends Enum>, AsyncDispatcherConcurrent> concurrentEventDispatchers =
Maps.newHashMap();
private boolean exitOnDispatchException = false;
public AsyncDispatcher(String name) {
this(name, new LinkedBlockingQueue<Event>());
}
public AsyncDispatcher(String name, BlockingQueue<Event> eventQueue) {
super(name);
this.name = name;
this.eventQueue = eventQueue;
}
public Runnable createThread() {
return new Runnable() {
@Override
public void run() {
while (!stopped && !Thread.currentThread().isInterrupted()) {
drained = eventQueue.isEmpty();
// blockNewEvents is only set when dispatcher is draining to stop,
// adding this check is to avoid the overhead of acquiring the lock
// and calling notify every time in the normal run of the loop.
if (blockNewEvents) {
synchronized (waitForDrained) {
if (drained) {
waitForDrained.notify();
}
}
}
Event event;
try {
event = eventQueue.take();
if (LOG.isTraceEnabled()) {
LOG.trace("AsyncDispatcher taken event: {}", event);
}
} catch(InterruptedException ie) {
if (!stopped) {
LOG.warn("AsyncDispatcher thread interrupted (while taking event)", ie);
}
return;
}
if (event != null) {
dispatch(event);
}
}
}
};
}
@Override
protected void serviceInit(Configuration conf) throws Exception {
super.serviceInit(conf);
}
@Override
protected void serviceStart() throws Exception {
eventHandlingThread = new Thread(createThread());
eventHandlingThread.setName("Dispatcher thread {" + name + "}");
eventHandlingThread.start();
//start all the components
super.serviceStart();
}
public void setDrainEventsOnStop() {
drainEventsOnStop = true;
}
@Override
protected void serviceStop() throws Exception {
LOG.info("AsyncDispatcher serviceStop called, drainEventsOnStop: {}, drained: {}, eventQueue size: {}",
drainEventsOnStop, drained, eventQueue.size());
if (drainEventsOnStop) {
blockNewEvents = true;
LOG.info("AsyncDispatcher is draining to stop, ignoring any new events.");
long endTime = System.currentTimeMillis() + getConfig()
.getInt(TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT,
TezConfiguration.TEZ_AM_DISPATCHER_DRAIN_EVENTS_TIMEOUT_DEFAULT);
synchronized (waitForDrained) {
while (!eventQueue.isEmpty() && eventHandlingThread.isAlive() && System.currentTimeMillis() < endTime) {
waitForDrained.wait(1000);
LOG.info(
"Waiting for AsyncDispatcher to drain. Current queue size: {}, handler thread state: {}",
eventQueue.size(), eventHandlingThread.getState());
}
}
}
stopped = true;
if (eventHandlingThread != null) {
eventHandlingThread.interrupt();
try {
/*
* The event handling thread can be alive at this point, but in BLOCKED state, which leads
* to app hang, as a BLOCKED thread might never finish under some circumstances
*/
if (eventHandlingThread.getState() == Thread.State.BLOCKED) {
LOG.warn(
"eventHandlingThread is in BLOCKED state, let's not wait for it in order to prevent app hang");
} else {
eventHandlingThread.join();
LOG.info("joined event handling thread, state: {}", eventHandlingThread.getState());
}
} catch (InterruptedException ie) {
LOG.warn("Interrupted Exception while stopping", ie);
}
}
// stop all the components
super.serviceStop();
}
protected void dispatch(Event event) {
//all events go thru this loop
if (LOG.isDebugEnabled()) {
LOG.debug("Dispatching the event " + event.getClass().getName() + "."
+ event.toString());
}
Class<? extends Enum> type = event.getType().getDeclaringClass();
try{
EventHandler handler = eventHandlers.get(type);
if(handler != null) {
handler.handle(event);
} else {
throw new Exception("No handler for registered for " + type);
}
} catch (Throwable t) {
if (t instanceof InterruptedException) {
LOG.warn("Interrupted Exception while handling event: " + event.getType(), t);
Thread.currentThread().interrupt();
}
LOG.error("Error in dispatcher thread", t);
// If serviceStop is called, we should exit this thread gracefully.
if (exitOnDispatchException
&& (ShutdownHookManager.get().isShutdownInProgress()) == false
&& stopped == false) {
Thread shutDownThread = new Thread(createShutDownThread());
shutDownThread.setName("AsyncDispatcher ShutDown handler");
shutDownThread.start();
}
}
}
private void checkForExistingHandler(Class<? extends Enum> eventType) {
EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
Preconditions.checkState(registeredHandler == null,
"Cannot register same event on multiple dispatchers");
}
private void checkForExistingDispatcher(Class<? extends Enum> eventType) {
AsyncDispatcher registeredDispatcher = eventDispatchers.get(eventType);
Preconditions.checkState(registeredDispatcher == null,
"Multiple dispatchers cannot be registered for: " + eventType.getName());
}
private void checkForExistingConcurrentDispatcher(Class<? extends Enum> eventType) {
AsyncDispatcherConcurrent concurrentDispatcher = concurrentEventDispatchers.get(eventType);
Preconditions.checkState(concurrentDispatcher == null,
"Multiple concurrent dispatchers cannot be registered for: " + eventType.getName());
}
@VisibleForTesting
protected void checkForExistingDispatchers(boolean checkHandler, Class<? extends Enum> eventType) {
if (checkHandler) {
checkForExistingHandler(eventType);
}
checkForExistingDispatcher(eventType);
checkForExistingConcurrentDispatcher(eventType);
}
@VisibleForTesting
public void enableExitOnDispatchException() {
exitOnDispatchException = true;
}
/**
* Add an EventHandler for events handled inline on this dispatcher
*/
@Override
public void register(Class<? extends Enum> eventType,
EventHandler handler) {
Preconditions.checkState(getServiceState() == STATE.NOTINITED);
/* check to see if we have a listener registered */
EventHandler<Event> registeredHandler = (EventHandler<Event>) eventHandlers.get(eventType);
checkForExistingDispatchers(false, eventType);
LOG.info("Registering " + eventType + " for " + handler.getClass());
if (registeredHandler == null) {
eventHandlers.put(eventType, handler);
} else if (!(registeredHandler instanceof MultiListenerHandler)){
/* for multiple listeners of an event add the multiple listener handler */
MultiListenerHandler multiHandler = new MultiListenerHandler();
multiHandler.addHandler(registeredHandler);
multiHandler.addHandler(handler);
eventHandlers.put(eventType, multiHandler);
} else {
/* already a multilistener, just add to it */
MultiListenerHandler multiHandler
= (MultiListenerHandler) registeredHandler;
multiHandler.addHandler(handler);
}
}
/**
* Add an EventHandler for events handled in their own dispatchers with given name
*/
public void registerAndCreateDispatcher(Class<? extends Enum> eventType,
EventHandler handler, String dispatcherName) {
Preconditions.checkState(getServiceState() == STATE.NOTINITED);
/* check to see if we have a listener registered */
checkForExistingDispatchers(true, eventType);
LOG.info(
"Registering " + eventType + " for independent dispatch using: " + handler.getClass());
AsyncDispatcher dispatcher = new AsyncDispatcher(dispatcherName);
dispatcher.register(eventType, handler);
eventDispatchers.put(eventType, dispatcher);
addIfService(dispatcher);
}
public AsyncDispatcherConcurrent registerAndCreateDispatcher(Class<? extends Enum> eventType,
EventHandler handler, String dispatcherName, int numThreads) {
Preconditions.checkState(getServiceState() == STATE.NOTINITED);
/* check to see if we have a listener registered */
checkForExistingDispatchers(true, eventType);
LOG.info(
"Registering " + eventType + " for concurrent dispatch using: " + handler.getClass());
AsyncDispatcherConcurrent dispatcher = new AsyncDispatcherConcurrent(dispatcherName, numThreads);
if (exitOnDispatchException) {
dispatcher.enableExitOnDispatchException();
}
dispatcher.register(eventType, handler);
concurrentEventDispatchers.put(eventType, dispatcher);
addIfService(dispatcher);
return dispatcher;
}
public void registerWithExistingDispatcher(Class<? extends Enum> eventType,
EventHandler handler, AsyncDispatcherConcurrent dispatcher) {
Preconditions.checkState(getServiceState() == STATE.NOTINITED);
/* check to see if we have a listener registered */
checkForExistingDispatchers(true, eventType);
LOG.info("Registering " + eventType + " with existing concurrent dispatch using: "
+ handler.getClass());
if (exitOnDispatchException) {
dispatcher.enableExitOnDispatchException();
}
dispatcher.register(eventType, handler);
concurrentEventDispatchers.put(eventType, dispatcher);
}
@Override
public EventHandler getEventHandler() {
return handlerInstance;
}
class GenericEventHandler implements EventHandler<Event> {
public void handle(Event event) {
if (stopped) {
return;
}
if (blockNewEvents) {
return;
}
drained = false;
// offload to specific dispatcher if one exists
Class<? extends Enum> type = event.getType().getDeclaringClass();
AsyncDispatcher registeredDispatcher = eventDispatchers.get(type);
if (registeredDispatcher != null) {
registeredDispatcher.getEventHandler().handle(event);
return;
}
AsyncDispatcherConcurrent concurrentDispatcher = concurrentEventDispatchers.get(type);
if (concurrentDispatcher != null) {
concurrentDispatcher.getEventHandler().handle(event);
return;
}
// no registered dispatcher. use internal dispatcher.
/* all this method does is enqueue all the events onto the queue */
int qSize = eventQueue.size();
if (qSize !=0 && qSize %1000 == 0) {
LOG.info("Size of event-queue is " + qSize);
}
int remCapacity = eventQueue.remainingCapacity();
if (remCapacity < 1000) {
LOG.warn("Very low remaining capacity in the event-queue: "
+ remCapacity);
}
try {
eventQueue.put(event);
if (LOG.isTraceEnabled()) {
LOG.trace("AsyncDispatcher put event: {}", event);
}
} catch (InterruptedException e) {
if (!stopped) {
LOG.warn("AsyncDispatcher thread interrupted (while putting event): {}", event, e);
}
throw new YarnRuntimeException(e);
}
};
}
/**
* Multiplexing an event. Sending it to different handlers that
* are interested in the event.
* @param <T> the type of event these multiple handlers are interested in.
*/
static class MultiListenerHandler implements EventHandler<Event> {
List<EventHandler<Event>> listofHandlers;
public MultiListenerHandler() {
listofHandlers = new ArrayList<EventHandler<Event>>();
}
@Override
public void handle(Event event) {
for (EventHandler<Event> handler: listofHandlers) {
handler.handle(event);
}
}
void addHandler(EventHandler<Event> handler) {
listofHandlers.add(handler);
}
}
Runnable createShutDownThread() {
return new Runnable() {
@Override
public void run() {
LOG.info("Exiting, bbye..");
System.exit(-1);
}
};
}
@Private
public int getQueueSize() {
return eventQueue.size();
}
}