001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    
019    package org.apache.hadoop.yarn.event;
020    
021    import java.util.ArrayList;
022    import java.util.HashMap;
023    import java.util.List;
024    import java.util.Map;
025    import java.util.concurrent.BlockingQueue;
026    import java.util.concurrent.LinkedBlockingQueue;
027    
028    import org.apache.commons.logging.Log;
029    import org.apache.commons.logging.LogFactory;
030    import org.apache.hadoop.classification.InterfaceAudience.Public;
031    import org.apache.hadoop.classification.InterfaceStability.Evolving;
032    import org.apache.hadoop.conf.Configuration;
033    import org.apache.hadoop.service.AbstractService;
034    import org.apache.hadoop.util.ShutdownHookManager;
035    import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
036    
037    /**
038     * Dispatches {@link Event}s in a separate thread. Currently only single thread
039     * does that. Potentially there could be multiple channels for each event type
040     * class and a thread pool can be used to dispatch the events.
041     */
042    @SuppressWarnings("rawtypes")
043    @Public
044    @Evolving
045    public class AsyncDispatcher extends AbstractService implements Dispatcher {
046    
047      private static final Log LOG = LogFactory.getLog(AsyncDispatcher.class);
048    
049      private final BlockingQueue<Event> eventQueue;
050      private volatile boolean stopped = false;
051    
052      private Thread eventHandlingThread;
053      protected final Map<Class<? extends Enum>, EventHandler> eventDispatchers;
054      private boolean exitOnDispatchException;
055    
056      public AsyncDispatcher() {
057        this(new LinkedBlockingQueue<Event>());
058      }
059    
060      public AsyncDispatcher(BlockingQueue<Event> eventQueue) {
061        super("Dispatcher");
062        this.eventQueue = eventQueue;
063        this.eventDispatchers = new HashMap<Class<? extends Enum>, EventHandler>();
064      }
065    
066      Runnable createThread() {
067        return new Runnable() {
068          @Override
069          public void run() {
070            while (!stopped && !Thread.currentThread().isInterrupted()) {
071              Event event;
072              try {
073                event = eventQueue.take();
074              } catch(InterruptedException ie) {
075                if (!stopped) {
076                  LOG.warn("AsyncDispatcher thread interrupted", ie);
077                }
078                return;
079              }
080              if (event != null) {
081                dispatch(event);
082              }
083            }
084          }
085        };
086      }
087    
088      @Override
089      protected void serviceInit(Configuration conf) throws Exception {
090        this.exitOnDispatchException =
091            conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
092              Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
093        super.serviceInit(conf);
094      }
095    
096      @Override
097      protected void serviceStart() throws Exception {
098        //start all the components
099        super.serviceStart();
100        eventHandlingThread = new Thread(createThread());
101        eventHandlingThread.setName("AsyncDispatcher event handler");
102        eventHandlingThread.start();
103      }
104    
105      @Override
106      protected void serviceStop() throws Exception {
107        stopped = true;
108        if (eventHandlingThread != null) {
109          eventHandlingThread.interrupt();
110          try {
111            eventHandlingThread.join();
112          } catch (InterruptedException ie) {
113            LOG.warn("Interrupted Exception while stopping", ie);
114          }
115        }
116    
117        // stop all the components
118        super.serviceStop();
119      }
120    
121      @SuppressWarnings("unchecked")
122      protected void dispatch(Event event) {
123        //all events go thru this loop
124        if (LOG.isDebugEnabled()) {
125          LOG.debug("Dispatching the event " + event.getClass().getName() + "."
126              + event.toString());
127        }
128    
129        Class<? extends Enum> type = event.getType().getDeclaringClass();
130    
131        try{
132          EventHandler handler = eventDispatchers.get(type);
133          if(handler != null) {
134            handler.handle(event);
135          } else {
136            throw new Exception("No handler for registered for " + type);
137          }
138        }
139        catch (Throwable t) {
140          //TODO Maybe log the state of the queue
141          LOG.fatal("Error in dispatcher thread", t);
142          if (exitOnDispatchException
143              && (ShutdownHookManager.get().isShutdownInProgress()) == false) {
144            LOG.info("Exiting, bbye..");
145            System.exit(-1);
146          }
147        }
148      }
149    
150      @SuppressWarnings("unchecked")
151      @Override
152      public void register(Class<? extends Enum> eventType,
153          EventHandler handler) {
154        /* check to see if we have a listener registered */
155        EventHandler<Event> registeredHandler = (EventHandler<Event>)
156        eventDispatchers.get(eventType);
157        LOG.info("Registering " + eventType + " for " + handler.getClass());
158        if (registeredHandler == null) {
159          eventDispatchers.put(eventType, handler);
160        } else if (!(registeredHandler instanceof MultiListenerHandler)){
161          /* for multiple listeners of an event add the multiple listener handler */
162          MultiListenerHandler multiHandler = new MultiListenerHandler();
163          multiHandler.addHandler(registeredHandler);
164          multiHandler.addHandler(handler);
165          eventDispatchers.put(eventType, multiHandler);
166        } else {
167          /* already a multilistener, just add to it */
168          MultiListenerHandler multiHandler
169          = (MultiListenerHandler) registeredHandler;
170          multiHandler.addHandler(handler);
171        }
172      }
173    
174      @Override
175      public EventHandler getEventHandler() {
176        return new GenericEventHandler();
177      }
178    
179      class GenericEventHandler implements EventHandler<Event> {
180        public void handle(Event event) {
181          /* all this method does is enqueue all the events onto the queue */
182          int qSize = eventQueue.size();
183          if (qSize !=0 && qSize %1000 == 0) {
184            LOG.info("Size of event-queue is " + qSize);
185          }
186          int remCapacity = eventQueue.remainingCapacity();
187          if (remCapacity < 1000) {
188            LOG.warn("Very low remaining capacity in the event-queue: "
189                + remCapacity);
190          }
191          try {
192            eventQueue.put(event);
193          } catch (InterruptedException e) {
194            if (!stopped) {
195              LOG.warn("AsyncDispatcher thread interrupted", e);
196            }
197            throw new YarnRuntimeException(e);
198          }
199        };
200      }
201    
202      /**
203       * Multiplexing an event. Sending it to different handlers that
204       * are interested in the event.
205       * @param <T> the type of event these multiple handlers are interested in.
206       */
207      static class MultiListenerHandler implements EventHandler<Event> {
208        List<EventHandler<Event>> listofHandlers;
209    
210        public MultiListenerHandler() {
211          listofHandlers = new ArrayList<EventHandler<Event>>();
212        }
213    
214        @Override
215        public void handle(Event event) {
216          for (EventHandler<Event> handler: listofHandlers) {
217            handler.handle(event);
218          }
219        }
220    
221        void addHandler(EventHandler<Event> handler) {
222          listofHandlers.add(handler);
223        }
224    
225      }
226    }