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.util.ArrayList;
23  import java.util.Collections;
24  import java.util.List;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.hbase.executor.HBaseExecutorService.HBaseExecutorServiceType;
29  import org.apache.hadoop.hbase.master.ServerManager;
30  
31  
32  /**
33   * Abstract base class for all HBase event handlers. Subclasses should 
34   * implement the process() method where the actual handling of the event 
35   * happens.
36   * 
37   * HBaseEventType is a list of ALL events (which also corresponds to messages - 
38   * either internal to one component or between components). The event type 
39   * names specify the component from which the event originated, and the 
40   * component which is supposed to handle it.
41   * 
42   * Listeners can listen to all the events by implementing the interface 
43   * HBaseEventHandlerListener, and by registering themselves as a listener. They 
44   * will be called back before and after the process of every event.
45   * 
46   * TODO: Rename HBaseEvent and HBaseEventType to EventHandler and EventType 
47   * after ZK refactor as it currently would clash with EventType from ZK and 
48   * make the code very confusing.
49   */
50  public abstract class HBaseEventHandler implements Runnable
51  {
52    private static final Log LOG = LogFactory.getLog(HBaseEventHandler.class);
53    // type of event this object represents
54    protected HBaseEventType eventType = HBaseEventType.NONE;
55    // is this a region server or master?
56    protected boolean isRegionServer;
57    // name of the server - this is needed for naming executors in case of tests 
58    // where region servers may be co-located.
59    protected String serverName;
60    // listeners that are called before and after an event is processed
61    protected static List<HBaseEventHandlerListener> eventHandlerListeners = 
62      Collections.synchronizedList(new ArrayList<HBaseEventHandlerListener>());  
63  
64    /**
65     * This interface provides hooks to listen to various events received by the 
66     * queue. A class implementing this can listen to the updates by calling 
67     * registerListener and stop receiving updates by calling unregisterListener
68     */
69    public interface HBaseEventHandlerListener {
70      /**
71       * Called before any event is processed
72       */
73      public void beforeProcess(HBaseEventHandler event);
74      /**
75       * Called after any event is processed
76       */
77      public void afterProcess(HBaseEventHandler event);
78    }
79  
80    /**
81     * These are a list of HBase events that can be handled by the various
82     * HBaseExecutorService's. All the events are serialized as byte values.
83     */
84    public enum HBaseEventType {
85      NONE (-1),
86      // Messages originating from RS (NOTE: there is NO direct communication from 
87      // RS to Master). These are a result of RS updates into ZK.
88      RS2ZK_REGION_CLOSING      (1),   // RS is in process of closing a region
89      RS2ZK_REGION_CLOSED       (2),   // RS has finished closing a region
90      RS2ZK_REGION_OPENING      (3),   // RS is in process of opening a region
91      RS2ZK_REGION_OPENED       (4),   // RS has finished opening a region
92      
93      // Updates from master to ZK. This is done by the master and there is 
94      // nothing to process by either Master or RS
95      M2ZK_REGION_OFFLINE       (50);  // Master adds this region as offline in ZK
96      
97      private final byte value;
98      
99      /**
100      * Called by the HMaster. Returns a name of the executor service given an 
101      * event type. Every event type has en entry - if the event should not be 
102      * handled just add the NONE executor.
103      * @return name of the executor service
104      */
105     public HBaseExecutorServiceType getMasterExecutorForEvent() {
106       HBaseExecutorServiceType executorServiceType = null;
107       switch(this) {
108       
109       case RS2ZK_REGION_CLOSING:
110       case RS2ZK_REGION_CLOSED:
111         executorServiceType = HBaseExecutorServiceType.MASTER_CLOSEREGION;
112         break;
113 
114       case RS2ZK_REGION_OPENING:
115       case RS2ZK_REGION_OPENED:
116         executorServiceType = HBaseExecutorServiceType.MASTER_OPENREGION;
117         break;
118         
119       case M2ZK_REGION_OFFLINE:
120         executorServiceType = HBaseExecutorServiceType.NONE;
121         break;
122         
123       default:
124         throw new RuntimeException("Unhandled event type in the master.");
125       }
126       
127       return executorServiceType;
128     }
129 
130     /**
131      * Called by the RegionServer. Returns a name of the executor service given an 
132      * event type. Every event type has en entry - if the event should not be 
133      * handled just return a null executor name.
134      * @return name of the event service
135      */
136     public static String getRSExecutorForEvent(String serverName) {
137       throw new RuntimeException("Unsupported operation.");
138     }
139     
140     /**
141      * Start the executor service that handles the passed in event type. The 
142      * server that starts these event executor services wants to handle these 
143      * event types.
144      */
145     public void startMasterExecutorService(String serverName) {
146       HBaseExecutorServiceType serviceType = getMasterExecutorForEvent();
147       if(serviceType == HBaseExecutorServiceType.NONE) {
148         throw new RuntimeException("Event type " + toString() + " not handled on master.");
149       }
150       serviceType.startExecutorService(serverName);
151     }
152 
153     public static void startRSExecutorService() {
154       
155     }
156 
157     HBaseEventType(int intValue) {
158       this.value = (byte)intValue;
159     }
160     
161     public byte getByteValue() {
162       return value;
163     }
164 
165     public static HBaseEventType fromByte(byte value) {
166       switch(value) {
167         case  -1: return HBaseEventType.NONE;
168         case  1 : return HBaseEventType.RS2ZK_REGION_CLOSING;
169         case  2 : return HBaseEventType.RS2ZK_REGION_CLOSED;
170         case  3 : return HBaseEventType.RS2ZK_REGION_OPENING;
171         case  4 : return HBaseEventType.RS2ZK_REGION_OPENED;
172         case  50: return HBaseEventType.M2ZK_REGION_OFFLINE;
173 
174         default:
175           throw new RuntimeException("Invalid byte value for conversion to HBaseEventType");
176       }
177     }
178   }
179   
180   /**
181    * Default base class constructor.
182    * 
183    * TODO: isRegionServer and serverName will go away once we do the HMaster 
184    * refactor. We will end up passing a ServerStatus which should tell us both 
185    * the name and if it is a RS or master.
186    */
187   public HBaseEventHandler(boolean isRegionServer, String serverName, HBaseEventType eventType) {
188     this.isRegionServer = isRegionServer;
189     this.eventType = eventType;
190     this.serverName = serverName;
191   }
192   
193   /**
194    * This is a wrapper around process, used to update listeners before and after 
195    * events are processed. 
196    */
197   public void run() {
198     // fire all beforeProcess listeners
199     for(HBaseEventHandlerListener listener : eventHandlerListeners) {
200       listener.beforeProcess(this);
201     }
202     
203     // call the main process function
204     try {
205       process();
206     } catch(Throwable t) {
207       LOG.error("Caught throwable while processing event " + eventType, t);
208     }
209 
210     // fire all afterProcess listeners
211     for(HBaseEventHandlerListener listener : eventHandlerListeners) {
212       LOG.debug("Firing " + listener.getClass().getName() + 
213                 ".afterProcess event listener for event " + eventType);
214       listener.afterProcess(this);
215     }
216   }
217   
218   /**
219    * This method is the main processing loop to be implemented by the various 
220    * subclasses.
221    */
222   public abstract void process();
223   
224   /**
225    * Subscribe to updates before and after processing events
226    */
227   public static void registerListener(HBaseEventHandlerListener listener) {
228     eventHandlerListeners.add(listener);
229   }
230   
231   /**
232    * Stop receiving updates before and after processing events
233    */
234   public static void unregisterListener(HBaseEventHandlerListener listener) {
235     eventHandlerListeners.remove(listener);
236   }
237   
238   public boolean isRegionServer() {
239     return isRegionServer;
240   }
241 
242   /**
243    * Return the name for this event type.
244    * @return
245    */
246   public HBaseExecutorServiceType getEventHandlerName() {
247     // TODO: check for isRegionServer here
248     return eventType.getMasterExecutorForEvent();
249   }
250   
251   /**
252    * Return the event type
253    * @return
254    */
255   public HBaseEventType getHBEvent() {
256     return eventType;
257   }
258 
259   /**
260    * Submits this event object to the correct executor service. This is causes
261    * this object to get executed by the correct ExecutorService.
262    */
263   public void submit() {
264     HBaseExecutorServiceType serviceType = getEventHandlerName();
265     if(serviceType == null) {
266       throw new RuntimeException("Event " + eventType + " not handled on this server " + serverName);
267     }
268     serviceType.getExecutor(serverName).submit(this);
269   }
270   
271   /**
272    * Executes this event object in the caller's thread. This is a synchronous 
273    * way of executing the event.
274    */
275   public void execute() {
276     this.run();
277   }
278 }