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 */
018package org.apache.hadoop.hbase.executor;
019
020import org.apache.yetus.audience.InterfaceAudience;
021
022/**
023 * List of all HBase event handler types.
024 * <p>
025 * Event types are named by a convention: event type names specify the component from which the
026 * event originated and then where its destined -- e.g. RS_ZK_ prefix means the event came from a
027 * regionserver destined for zookeeper -- and then what the even is; e.g. REGION_OPENING.
028 * <p>
029 * We give the enums indices so we can add types later and keep them grouped together rather than
030 * have to add them always to the end as we would have to if we used raw enum ordinals.
031 */
032@InterfaceAudience.Private
033public enum EventType {
034  // Messages originating from RS (NOTE: there is NO direct communication from
035  // RS to Master). These are a result of RS updates into ZK.
036  // RS_ZK_REGION_CLOSING    (1),   // It is replaced by M_ZK_REGION_CLOSING(HBASE-4739)
037
038  /**
039   * RS_ZK_REGION_CLOSED<br>
040   *
041   * RS has finished closing a region.
042   */
043  RS_ZK_REGION_CLOSED       (2, ExecutorType.MASTER_CLOSE_REGION),
044  /**
045   * RS_ZK_REGION_OPENING<br>
046   *
047   * RS is in process of opening a region.
048   */
049  RS_ZK_REGION_OPENING      (3, null),
050  /**
051   * RS_ZK_REGION_OPENED<br>
052   *
053   * RS has finished opening a region.
054   */
055  RS_ZK_REGION_OPENED       (4, ExecutorType.MASTER_OPEN_REGION),
056  /**
057   * RS_ZK_REGION_SPLITTING<br>
058   *
059   * RS has started a region split after master says it's ok to move on.
060   */
061  RS_ZK_REGION_SPLITTING    (5, null),
062  /**
063   * RS_ZK_REGION_SPLIT<br>
064   *
065   * RS split has completed and is notifying the master.
066   */
067  RS_ZK_REGION_SPLIT        (6, ExecutorType.MASTER_SERVER_OPERATIONS),
068  /**
069   * RS_ZK_REGION_FAILED_OPEN<br>
070   *
071   * RS failed to open a region.
072   */
073  RS_ZK_REGION_FAILED_OPEN  (7, ExecutorType.MASTER_CLOSE_REGION),
074  /**
075   * RS_ZK_REGION_MERGING<br>
076   *
077   * RS has started merging regions after master says it's ok to move on.
078   */
079  RS_ZK_REGION_MERGING      (8, null),
080  /**
081   * RS_ZK_REGION_MERGE<br>
082   *
083   * RS region merge has completed and is notifying the master.
084   */
085  RS_ZK_REGION_MERGED       (9, ExecutorType.MASTER_SERVER_OPERATIONS),
086  /**
087   * RS_ZK_REQUEST_REGION_SPLIT<br>
088   *
089   * RS has requested to split a region. This is to notify master
090   * and check with master if the region is in a state good to split.
091   */
092  RS_ZK_REQUEST_REGION_SPLIT    (10, null),
093  /**
094   * RS_ZK_REQUEST_REGION_MERGE<br>
095   *
096   * RS has requested to merge two regions. This is to notify master
097   * and check with master if two regions is in states good to merge.
098   */
099  RS_ZK_REQUEST_REGION_MERGE    (11, null),
100
101  /**
102   * Messages originating from Master to RS.<br>
103   * M_RS_OPEN_REGION<br>
104   * Master asking RS to open a region.
105   */
106  M_RS_OPEN_REGION          (20, ExecutorType.RS_OPEN_REGION),
107  /**
108   * Messages originating from Master to RS.<br>
109   * M_RS_OPEN_ROOT<br>
110   * Master asking RS to open root.
111   */
112  M_RS_OPEN_ROOT            (21, ExecutorType.RS_OPEN_ROOT),
113  /**
114   * Messages originating from Master to RS.<br>
115   * M_RS_OPEN_META<br>
116   * Master asking RS to open meta.
117   */
118  M_RS_OPEN_META            (22, ExecutorType.RS_OPEN_META),
119  /**
120   * Messages originating from Master to RS.<br>
121   * M_RS_CLOSE_REGION<br>
122   * Master asking RS to close a region.
123   */
124  M_RS_CLOSE_REGION         (23, ExecutorType.RS_CLOSE_REGION),
125  /**
126   * Messages originating from Master to RS.<br>
127   * M_RS_CLOSE_ROOT<br>
128   * Master asking RS to close root.
129   */
130  M_RS_CLOSE_ROOT           (24, ExecutorType.RS_CLOSE_ROOT),
131  /**
132   * Messages originating from Master to RS.<br>
133   * M_RS_CLOSE_META<br>
134   * Master asking RS to close meta.
135   */
136  M_RS_CLOSE_META           (25, ExecutorType.RS_CLOSE_META),
137  /**
138   * Messages originating from Master to RS.<br>
139   * M_RS_OPEN_PRIORITY_REGION<br>
140   * Master asking RS to open a  priority region.
141   */
142  M_RS_OPEN_PRIORITY_REGION          (26, ExecutorType.RS_OPEN_PRIORITY_REGION),
143  /**
144   * Messages originating from Master to RS.<br>
145   * M_RS_SWITCH_RPC_THROTTLE<br>
146   * Master asking RS to switch rpc throttle state.
147   */
148  M_RS_SWITCH_RPC_THROTTLE(27, ExecutorType.RS_SWITCH_RPC_THROTTLE),
149
150  /**
151   * Messages originating from Client to Master.<br>
152   * C_M_MERGE_REGION<br>
153   * Client asking Master to merge regions.
154   */
155  C_M_MERGE_REGION          (30, ExecutorType.MASTER_TABLE_OPERATIONS),
156  /**
157   * Messages originating from Client to Master.<br>
158   * C_M_DELETE_TABLE<br>
159   * Client asking Master to delete a table.
160   */
161  C_M_DELETE_TABLE          (40, ExecutorType.MASTER_TABLE_OPERATIONS),
162  /**
163   * Messages originating from Client to Master.<br>
164   * C_M_DISABLE_TABLE<br>
165   * Client asking Master to disable a table.
166   */
167  C_M_DISABLE_TABLE         (41, ExecutorType.MASTER_TABLE_OPERATIONS),
168  /**
169   * Messages originating from Client to Master.<br>
170   * C_M_ENABLE_TABLE<br>
171   * Client asking Master to enable a table.
172   */
173  C_M_ENABLE_TABLE          (42, ExecutorType.MASTER_TABLE_OPERATIONS),
174  /**
175   * Messages originating from Client to Master.<br>
176   * C_M_MODIFY_TABLE<br>
177   * Client asking Master to modify a table.
178   */
179  C_M_MODIFY_TABLE          (43, ExecutorType.MASTER_TABLE_OPERATIONS),
180  /**
181   * Messages originating from Client to Master.<br>
182   * C_M_ADD_FAMILY<br>
183   * Client asking Master to add family to table.
184   */
185  C_M_ADD_FAMILY            (44, null),
186  /**
187   * Messages originating from Client to Master.<br>
188   * C_M_DELETE_FAMILY<br>
189   * Client asking Master to delete family of table.
190   */
191  C_M_DELETE_FAMILY         (45, null),
192  /**
193   * Messages originating from Client to Master.<br>
194   * C_M_MODIFY_FAMILY<br>
195   * Client asking Master to modify family of table.
196   */
197  C_M_MODIFY_FAMILY         (46, null),
198  /**
199   * Messages originating from Client to Master.<br>
200   * C_M_CREATE_TABLE<br>
201   * Client asking Master to create a table.
202   */
203  C_M_CREATE_TABLE          (47, ExecutorType.MASTER_TABLE_OPERATIONS),
204  /**
205   * Messages originating from Client to Master.<br>
206   * C_M_SNAPSHOT_TABLE<br>
207   * Client asking Master to snapshot an offline table.
208   */
209  C_M_SNAPSHOT_TABLE        (48, ExecutorType.MASTER_SNAPSHOT_OPERATIONS),
210  /**
211   * Messages originating from Client to Master.<br>
212   * C_M_RESTORE_SNAPSHOT<br>
213   * Client asking Master to restore a snapshot.
214   */
215  C_M_RESTORE_SNAPSHOT      (49, ExecutorType.MASTER_SNAPSHOT_OPERATIONS),
216
217  // Updates from master to ZK. This is done by the master and there is
218  // nothing to process by either Master or RS
219  /**
220   * M_ZK_REGION_OFFLINE
221   * Master adds this region as offline in ZK
222   */
223  M_ZK_REGION_OFFLINE       (50, null),
224  /**
225   * M_ZK_REGION_CLOSING
226   * Master adds this region as closing in ZK
227   */
228  M_ZK_REGION_CLOSING       (51, null),
229
230  /**
231   * Master controlled events to be executed on the master
232   * M_SERVER_SHUTDOWN
233   * Master is processing shutdown of a RS
234   */
235  M_SERVER_SHUTDOWN         (70, ExecutorType.MASTER_SERVER_OPERATIONS),
236  /**
237   * Master controlled events to be executed on the master.<br>
238   * M_META_SERVER_SHUTDOWN <br>
239   * Master is processing shutdown of RS hosting a meta region (-ROOT- or hbase:meta).
240   */
241  M_META_SERVER_SHUTDOWN    (72, ExecutorType.MASTER_META_SERVER_OPERATIONS),
242  /**
243   * Master controlled events to be executed on the master.<br>
244   *
245   * M_MASTER_RECOVERY<br>
246   * Master is processing recovery of regions found in ZK RIT
247   */
248  M_MASTER_RECOVERY         (73, ExecutorType.MASTER_SERVER_OPERATIONS),
249  /**
250   * Master controlled events to be executed on the master.<br>
251   *
252   * M_LOG_REPLAY<br>
253   * Master is processing log replay of failed region server
254   */
255  M_LOG_REPLAY              (74, ExecutorType.M_LOG_REPLAY_OPS),
256
257  /**
258   * RS controlled events to be executed on the RS.<br>
259   *
260   * RS_PARALLEL_SEEK
261   */
262  RS_PARALLEL_SEEK          (80, ExecutorType.RS_PARALLEL_SEEK),
263
264  /**
265   * RS wal recovery work items (splitting wals) to be executed on the RS.<br>
266   *
267   * RS_LOG_REPLAY
268   */
269  RS_LOG_REPLAY             (81, ExecutorType.RS_LOG_REPLAY_OPS),
270
271  /**
272   * RS flush triggering from secondary region replicas to primary region replica. <br>
273   *
274   * RS_REGION_REPLICA_FLUSH
275   */
276  RS_REGION_REPLICA_FLUSH   (82, ExecutorType.RS_REGION_REPLICA_FLUSH_OPS),
277
278  /**
279   * RS compacted files discharger <br>
280   *
281   * RS_COMPACTED_FILES_DISCHARGER
282   */
283  RS_COMPACTED_FILES_DISCHARGER (83, ExecutorType.RS_COMPACTED_FILES_DISCHARGER),
284
285  /**
286   * RS refresh peer.<br>
287   *
288   * RS_REFRESH_PEER
289   */
290  RS_REFRESH_PEER (84, ExecutorType.RS_REFRESH_PEER);
291
292  private final int code;
293  private final ExecutorType executor;
294
295  /**
296   * Constructor
297   */
298  EventType(final int code, final ExecutorType executor) {
299    this.code = code;
300    this.executor = executor;
301  }
302
303  public int getCode() {
304    return this.code;
305  }
306
307  public static EventType get(final int code) {
308    // Is this going to be slow? Its used rare but still...
309    for (EventType et : EventType.values()) {
310      if (et.getCode() == code) {
311        return et;
312      }
313    }
314    throw new IllegalArgumentException("Unknown code " + code);
315  }
316
317  ExecutorType getExecutorServiceType() {
318    return this.executor;
319  }
320}