View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
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   * This is a generic executor service. This component abstracts a
51   * threadpool, a queue to which {@link EventHandler.EventType}s can be submitted,
52   * and a <code>Runnable</code> that handles the object that is added to the queue.
53   *
54   * <p>In order to create a new service, create an instance of this class and
55   * then do: <code>instance.startExecutorService("myService");</code>.  When done
56   * call {@link #shutdown()}.
57   *
58   * <p>In order to use the service created above, call
59   * {@link #submit(EventHandler)}. Register pre- and post- processing listeners
60   * by registering your implementation of {@link EventHandler.EventHandlerListener}
61   * with {@link #registerListener(EventHandler.EventType, EventHandler.EventHandlerListener)}.  Be sure
62   * to deregister your listener when done via {@link #unregisterListener(EventHandler.EventType)}.
63   */
64  public class ExecutorService {
65    private static final Log LOG = LogFactory.getLog(ExecutorService.class);
66  
67    // hold the all the executors created in a map addressable by their names
68    private final ConcurrentHashMap<String, Executor> executorMap =
69      new ConcurrentHashMap<String, Executor>();
70  
71    // listeners that are called before and after an event is processed
72    private ConcurrentHashMap<EventHandler.EventType, EventHandlerListener> eventHandlerListeners =
73      new ConcurrentHashMap<EventHandler.EventType, EventHandlerListener>();
74  
75    // Name of the server hosting this executor service.
76    private final String servername;
77  
78    /**
79     * The following is a list of all executor types, both those that run in the
80     * master and those that run in the regionserver.
81     */
82    public enum ExecutorType {
83  
84      // Master executor services
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      // RegionServer executor services
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      * @param serverName
104      * @return Conflation of the executor type and the passed servername.
105      */
106     String getExecutorName(String serverName) {
107       return this.toString() + "-" + serverName;
108     }
109   }
110 
111   /**
112    * Returns the executor service type (the thread pool instance) for the
113    * passed event handler type.
114    * @param type EventHandler type.
115    */
116   public ExecutorType getExecutorServiceType(final EventHandler.EventType type) {
117     switch(type) {
118       // Master executor services
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       // RegionServer executor services
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    * Default constructor.
168    * @param servername Name of the hosting server.
169    */
170   public ExecutorService(final String servername) {
171     super();
172     this.servername = servername;
173   }
174 
175   /**
176    * Start an executor service with a given name. If there was a service already
177    * started with the same name, this throws a RuntimeException.
178    * @param name Name of the service to start.
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       // This happens only when events are submitted after shutdown() was
234       // called, so dropping them should be "ok" since it means we're
235       // shutting down.
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    * Subscribe to updates before and after processing instances of
245    * {@link EventHandler.EventType}.  Currently only one listener per
246    * event type.
247    * @param type Type of event we're registering listener for
248    * @param listener The listener to run.
249    */
250   public void registerListener(final EventHandler.EventType type,
251       final EventHandlerListener listener) {
252     this.eventHandlerListeners.put(type, listener);
253   }
254 
255   /**
256    * Stop receiving updates before and after processing instances of
257    * {@link EventHandler.EventType}
258    * @param type Type of event we're registering listener for
259    * @return The listener we removed or null if we did not remove it.
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    * Executor instance.
275    */
276   static class Executor {
277     // how long to retain excess threads
278     final long keepAliveTimeInMillis = 1000;
279     // the thread pool executor that services the requests
280     final TrackingThreadPoolExecutor threadPoolExecutor;
281     // work queue to use - unbounded queue
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       // create the thread pool executor
294       this.threadPoolExecutor = new TrackingThreadPoolExecutor(
295           maxThreads, maxThreads,
296           keepAliveTimeInMillis, TimeUnit.MILLISECONDS, q);
297       // name the threads for this threadpool
298       ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
299       tfb.setNameFormat(this.name + "-%d");
300       this.threadPoolExecutor.setThreadFactory(tfb.build());
301     }
302 
303     /**
304      * Submit the event to the queue for handling.
305      * @param event
306      */
307     void submit(final EventHandler event) {
308       // If there is a listener for this type, make sure we call the before
309       // and after process methods.
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    * A subclass of ThreadPoolExecutor that keeps track of the Runnables that
349    * are executing at any given point in time.
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      * @return a map of the threads currently running tasks
374      * inside this executor. Each key is an active thread,
375      * and the value is the task that is currently running.
376      * Note that this is not a stable snapshot of the map.
377      */
378     public ConcurrentMap<Thread, Runnable> getRunningTasks() {
379       return running;
380     }
381   }
382 
383   /**
384    * A snapshot of the status of a particular executor. This includes
385    * the contents of the executor's pending queue, as well as the
386    * threads and events currently being processed.
387    *
388    * This is a consistent snapshot that is immutable once constructed.
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      * Dump a textual representation of the executor's status
405      * to the given writer.
406      *
407      * @param out the stream to write to
408      * @param indent a string prefix for each line, used for indentation
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    * The status of a particular event that is in the middle of being
439    * handled by an executor.
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 }