1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.executor;
21
22 import java.io.IOException;
23 import java.io.PrintWriter;
24 import java.io.Writer;
25 import java.lang.management.ThreadInfo;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Map.Entry;
29 import java.util.concurrent.BlockingQueue;
30 import java.util.concurrent.ConcurrentHashMap;
31 import java.util.concurrent.ConcurrentMap;
32 import java.util.concurrent.LinkedBlockingQueue;
33 import java.util.concurrent.RejectedExecutionHandler;
34 import java.util.concurrent.ThreadFactory;
35 import java.util.concurrent.ThreadPoolExecutor;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.atomic.AtomicLong;
38
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.hbase.executor.EventHandler.EventHandlerListener;
42 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
43 import org.apache.hadoop.hbase.monitoring.ThreadMonitoring;
44
45 import com.google.common.collect.Lists;
46 import com.google.common.collect.Maps;
47 import com.google.common.util.concurrent.ThreadFactoryBuilder;
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64 public class ExecutorService {
65 private static final Log LOG = LogFactory.getLog(ExecutorService.class);
66
67
68 private final ConcurrentHashMap<String, Executor> executorMap =
69 new ConcurrentHashMap<String, Executor>();
70
71
72 private ConcurrentHashMap<EventHandler.EventType, EventHandlerListener> eventHandlerListeners =
73 new ConcurrentHashMap<EventHandler.EventType, EventHandlerListener>();
74
75
76 private final String servername;
77
78
79
80
81
82 public enum ExecutorType {
83
84
85 MASTER_CLOSE_REGION (1),
86 MASTER_OPEN_REGION (2),
87 MASTER_SERVER_OPERATIONS (3),
88 MASTER_TABLE_OPERATIONS (4),
89 MASTER_RS_SHUTDOWN (5),
90 MASTER_META_SERVER_OPERATIONS (6),
91
92
93 RS_OPEN_REGION (20),
94 RS_OPEN_ROOT (21),
95 RS_OPEN_META (22),
96 RS_CLOSE_REGION (23),
97 RS_CLOSE_ROOT (24),
98 RS_CLOSE_META (25);
99
100 ExecutorType(int value) {}
101
102
103
104
105
106 String getExecutorName(String serverName) {
107 return this.toString() + "-" + serverName;
108 }
109 }
110
111
112
113
114
115
116 public ExecutorType getExecutorServiceType(final EventHandler.EventType type) {
117 switch(type) {
118
119
120 case RS_ZK_REGION_CLOSED:
121 case RS_ZK_REGION_FAILED_OPEN:
122 return ExecutorType.MASTER_CLOSE_REGION;
123
124 case RS_ZK_REGION_OPENED:
125 return ExecutorType.MASTER_OPEN_REGION;
126
127 case RS_ZK_REGION_SPLIT:
128 case M_SERVER_SHUTDOWN:
129 return ExecutorType.MASTER_SERVER_OPERATIONS;
130
131 case M_META_SERVER_SHUTDOWN:
132 return ExecutorType.MASTER_META_SERVER_OPERATIONS;
133
134 case C_M_DELETE_TABLE:
135 case C_M_DISABLE_TABLE:
136 case C_M_ENABLE_TABLE:
137 case C_M_MODIFY_TABLE:
138 case C_M_CREATE_TABLE:
139 return ExecutorType.MASTER_TABLE_OPERATIONS;
140
141
142
143 case M_RS_OPEN_REGION:
144 return ExecutorType.RS_OPEN_REGION;
145
146 case M_RS_OPEN_ROOT:
147 return ExecutorType.RS_OPEN_ROOT;
148
149 case M_RS_OPEN_META:
150 return ExecutorType.RS_OPEN_META;
151
152 case M_RS_CLOSE_REGION:
153 return ExecutorType.RS_CLOSE_REGION;
154
155 case M_RS_CLOSE_ROOT:
156 return ExecutorType.RS_CLOSE_ROOT;
157
158 case M_RS_CLOSE_META:
159 return ExecutorType.RS_CLOSE_META;
160
161 default:
162 throw new RuntimeException("Unhandled event type " + type);
163 }
164 }
165
166
167
168
169
170 public ExecutorService(final String servername) {
171 super();
172 this.servername = servername;
173 }
174
175
176
177
178
179
180 void startExecutorService(String name, int maxThreads) {
181 if (this.executorMap.get(name) != null) {
182 throw new RuntimeException("An executor service with the name " + name +
183 " is already running!");
184 }
185 Executor hbes = new Executor(name, maxThreads, this.eventHandlerListeners);
186 if (this.executorMap.putIfAbsent(name, hbes) != null) {
187 throw new RuntimeException("An executor service with the name " + name +
188 " is already running (2)!");
189 }
190 LOG.debug("Starting executor service name=" + name +
191 ", corePoolSize=" + hbes.threadPoolExecutor.getCorePoolSize() +
192 ", maxPoolSize=" + hbes.threadPoolExecutor.getMaximumPoolSize());
193 }
194
195 boolean isExecutorServiceRunning(String name) {
196 return this.executorMap.containsKey(name);
197 }
198
199 public void shutdown() {
200 for(Entry<String, Executor> entry: this.executorMap.entrySet()) {
201 List<Runnable> wasRunning =
202 entry.getValue().threadPoolExecutor.shutdownNow();
203 if (!wasRunning.isEmpty()) {
204 LOG.info(entry.getValue() + " had " + wasRunning + " on shutdown");
205 }
206 }
207 this.executorMap.clear();
208 }
209
210 Executor getExecutor(final ExecutorType type) {
211 return getExecutor(type.getExecutorName(this.servername));
212 }
213
214 Executor getExecutor(String name) {
215 Executor executor = this.executorMap.get(name);
216 return executor;
217 }
218
219
220 public void startExecutorService(final ExecutorType type, final int maxThreads) {
221 String name = type.getExecutorName(this.servername);
222 if (isExecutorServiceRunning(name)) {
223 LOG.debug("Executor service " + toString() + " already running on " +
224 this.servername);
225 return;
226 }
227 startExecutorService(name, maxThreads);
228 }
229
230 public void submit(final EventHandler eh) {
231 Executor executor = getExecutor(getExecutorServiceType(eh.getEventType()));
232 if (executor == null) {
233
234
235
236 LOG.error("Cannot submit [" + eh + "] because the executor is missing." +
237 " Is this process shutting down?");
238 } else {
239 executor.submit(eh);
240 }
241 }
242
243
244
245
246
247
248
249
250 public void registerListener(final EventHandler.EventType type,
251 final EventHandlerListener listener) {
252 this.eventHandlerListeners.put(type, listener);
253 }
254
255
256
257
258
259
260
261 public EventHandlerListener unregisterListener(final EventHandler.EventType type) {
262 return this.eventHandlerListeners.remove(type);
263 }
264
265 public Map<String, ExecutorStatus> getAllExecutorStatuses() {
266 Map<String, ExecutorStatus> ret = Maps.newHashMap();
267 for (Map.Entry<String, Executor> e : executorMap.entrySet()) {
268 ret.put(e.getKey(), e.getValue().getStatus());
269 }
270 return ret;
271 }
272
273
274
275
276 static class Executor {
277
278 final long keepAliveTimeInMillis = 1000;
279
280 final TrackingThreadPoolExecutor threadPoolExecutor;
281
282 final BlockingQueue<Runnable> q = new LinkedBlockingQueue<Runnable>();
283 private final String name;
284 private final Map<EventHandler.EventType, EventHandlerListener> eventHandlerListeners;
285 private static final AtomicLong seqids = new AtomicLong(0);
286 private final long id;
287
288 protected Executor(String name, int maxThreads,
289 final Map<EventHandler.EventType, EventHandlerListener> eventHandlerListeners) {
290 this.id = seqids.incrementAndGet();
291 this.name = name;
292 this.eventHandlerListeners = eventHandlerListeners;
293
294 this.threadPoolExecutor = new TrackingThreadPoolExecutor(
295 maxThreads, maxThreads,
296 keepAliveTimeInMillis, TimeUnit.MILLISECONDS, q);
297
298 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
299 tfb.setNameFormat(this.name + "-%d");
300 this.threadPoolExecutor.setThreadFactory(tfb.build());
301 }
302
303
304
305
306
307 void submit(final EventHandler event) {
308
309
310 EventHandlerListener listener =
311 this.eventHandlerListeners.get(event.getEventType());
312 if (listener != null) {
313 event.setListener(listener);
314 }
315 this.threadPoolExecutor.execute(event);
316 }
317
318 public String toString() {
319 return getClass().getSimpleName() + "-" + id + "-" + name;
320 }
321
322 public ExecutorStatus getStatus() {
323 List<EventHandler> queuedEvents = Lists.newArrayList();
324 for (Runnable r : q) {
325 if (!(r instanceof EventHandler)) {
326 LOG.warn("Non-EventHandler " + r + " queued in " + name);
327 continue;
328 }
329 queuedEvents.add((EventHandler)r);
330 }
331
332 List<RunningEventStatus> running = Lists.newArrayList();
333 for (Map.Entry<Thread, Runnable> e :
334 threadPoolExecutor.getRunningTasks().entrySet()) {
335 Runnable r = e.getValue();
336 if (!(r instanceof EventHandler)) {
337 LOG.warn("Non-EventHandler " + r + " running in " + name);
338 continue;
339 }
340 running.add(new RunningEventStatus(e.getKey(), (EventHandler)r));
341 }
342
343 return new ExecutorStatus(this, queuedEvents, running);
344 }
345 }
346
347
348
349
350
351 static class TrackingThreadPoolExecutor extends ThreadPoolExecutor {
352 private ConcurrentMap<Thread, Runnable> running = Maps.newConcurrentMap();
353
354 public TrackingThreadPoolExecutor(int corePoolSize, int maximumPoolSize,
355 long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue) {
356 super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue);
357 }
358
359 @Override
360 protected void afterExecute(Runnable r, Throwable t) {
361 super.afterExecute(r, t);
362 running.remove(Thread.currentThread());
363 }
364
365 @Override
366 protected void beforeExecute(Thread t, Runnable r) {
367 Runnable oldPut = running.put(t, r);
368 assert oldPut == null : "inconsistency for thread " + t;
369 super.beforeExecute(t, r);
370 }
371
372
373
374
375
376
377
378 public ConcurrentMap<Thread, Runnable> getRunningTasks() {
379 return running;
380 }
381 }
382
383
384
385
386
387
388
389
390 public static class ExecutorStatus {
391 final Executor executor;
392 final List<EventHandler> queuedEvents;
393 final List<RunningEventStatus> running;
394
395 ExecutorStatus(Executor executor,
396 List<EventHandler> queuedEvents,
397 List<RunningEventStatus> running) {
398 this.executor = executor;
399 this.queuedEvents = queuedEvents;
400 this.running = running;
401 }
402
403
404
405
406
407
408
409
410 public void dumpTo(Writer out, String indent) throws IOException {
411 out.write(indent + "Status for executor: " + executor + "\n");
412 out.write(indent + "=======================================\n");
413 out.write(indent + queuedEvents.size() + " events queued, " +
414 running.size() + " running\n");
415 if (!queuedEvents.isEmpty()) {
416 out.write(indent + "Queued:\n");
417 for (EventHandler e : queuedEvents) {
418 out.write(indent + " " + e + "\n");
419 }
420 out.write("\n");
421 }
422 if (!running.isEmpty()) {
423 out.write(indent + "Running:\n");
424 for (RunningEventStatus stat : running) {
425 out.write(indent + " Running on thread '" +
426 stat.threadInfo.getThreadName() +
427 "': " + stat.event + "\n");
428 out.write(ThreadMonitoring.formatThreadInfo(
429 stat.threadInfo, indent + " "));
430 out.write("\n");
431 }
432 }
433 out.flush();
434 }
435 }
436
437
438
439
440
441 public static class RunningEventStatus {
442 final ThreadInfo threadInfo;
443 final EventHandler event;
444
445 public RunningEventStatus(Thread t, EventHandler event) {
446 this.threadInfo = ThreadMonitoring.getThreadInfo(t);
447 this.event = event;
448 }
449 }
450 }