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
019package org.apache.hadoop.hbase.coprocessor;
020
021import java.io.IOException;
022import java.util.List;
023import java.util.Set;
024import org.apache.hadoop.hbase.ClusterMetrics;
025import org.apache.hadoop.hbase.HBaseInterfaceAudience;
026import org.apache.hadoop.hbase.MetaMutationAnnotation;
027import org.apache.hadoop.hbase.NamespaceDescriptor;
028import org.apache.hadoop.hbase.ServerName;
029import org.apache.hadoop.hbase.TableName;
030import org.apache.hadoop.hbase.client.MasterSwitchType;
031import org.apache.hadoop.hbase.client.Mutation;
032import org.apache.hadoop.hbase.client.RegionInfo;
033import org.apache.hadoop.hbase.client.SnapshotDescription;
034import org.apache.hadoop.hbase.client.TableDescriptor;
035import org.apache.hadoop.hbase.master.RegionPlan;
036import org.apache.hadoop.hbase.net.Address;
037import org.apache.hadoop.hbase.quotas.GlobalQuotaSettings;
038import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
039import org.apache.hadoop.hbase.security.access.Permission;
040import org.apache.hadoop.hbase.security.access.UserPermission;
041import org.apache.yetus.audience.InterfaceAudience;
042import org.apache.yetus.audience.InterfaceStability;
043
044
045/**
046 * Defines coprocessor hooks for interacting with operations on the
047 * {@link org.apache.hadoop.hbase.master.HMaster} process.
048 * <br><br>
049 *
050 * Since most implementations will be interested in only a subset of hooks, this class uses
051 * 'default' functions to avoid having to add unnecessary overrides. When the functions are
052 * non-empty, it's simply to satisfy the compiler by returning value of expected (non-void) type.
053 * It is done in a way that these default definitions act as no-op. So our suggestion to
054 * implementation would be to not call these 'default' methods from overrides.
055 * <br><br>
056 *
057 * <h3>Exception Handling</h3>
058 * For all functions, exception handling is done as follows:
059 * <ul>
060 *   <li>Exceptions of type {@link IOException} are reported back to client.</li>
061 *   <li>For any other kind of exception:
062 *     <ul>
063 *       <li>If the configuration {@link CoprocessorHost#ABORT_ON_ERROR_KEY} is set to true, then
064 *         the server aborts.</li>
065 *       <li>Otherwise, coprocessor is removed from the server and
066 *         {@link org.apache.hadoop.hbase.DoNotRetryIOException} is returned to the client.</li>
067 *     </ul>
068 *   </li>
069 * </ul>
070 */
071@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
072@InterfaceStability.Evolving
073public interface MasterObserver {
074
075  /**
076   * Called before we create the region infos for this table. Called as part of create table RPC
077   * call.
078   * @param ctx the environment to interact with the framework and master
079   * @param desc the TableDescriptor for the table
080   * @return the TableDescriptor used to create the table. Default is the one passed in. Return
081   *         {@code null} means cancel the creation.
082   */
083  default TableDescriptor preCreateTableRegionsInfos(
084      final ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc)
085      throws IOException {
086    return desc;
087  }
088
089  /**
090   * Called before a new table is created by
091   * {@link org.apache.hadoop.hbase.master.HMaster}.  Called as part of create
092   * table RPC call.
093   * @param ctx the environment to interact with the framework and master
094   * @param desc the TableDescriptor for the table
095   * @param regions the initial regions created for the table
096   */
097  default void preCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
098      TableDescriptor desc, RegionInfo[] regions) throws IOException {}
099
100  /**
101   * Called after the createTable operation has been requested.  Called as part
102   * of create table RPC call.
103   * @param ctx the environment to interact with the framework and master
104   * @param desc the TableDescriptor for the table
105   * @param regions the initial regions created for the table
106   */
107  default void postCreateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
108      TableDescriptor desc, RegionInfo[] regions) throws IOException {}
109
110  /**
111   * Called before a new table is created by
112   * {@link org.apache.hadoop.hbase.master.HMaster}.  Called as part of create
113   * table procedure and it is async to the create RPC call.
114   *
115   * @param ctx the environment to interact with the framework and master
116   * @param desc the TableDescriptor for the table
117   * @param regions the initial regions created for the table
118   */
119  default void preCreateTableAction(
120      final ObserverContext<MasterCoprocessorEnvironment> ctx,
121      final TableDescriptor desc,
122      final RegionInfo[] regions) throws IOException {}
123
124  /**
125   * Called after the createTable operation has been requested.  Called as part
126   * of create table RPC call.  Called as part of create table procedure and
127   * it is async to the create RPC call.
128   *
129   * @param ctx the environment to interact with the framework and master
130   * @param desc the TableDescriptor for the table
131   * @param regions the initial regions created for the table
132   */
133  default void postCompletedCreateTableAction(
134      final ObserverContext<MasterCoprocessorEnvironment> ctx,
135      final TableDescriptor desc,
136      final RegionInfo[] regions) throws IOException {}
137
138  /**
139   * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
140   * table.  Called as part of delete table RPC call.
141   * @param ctx the environment to interact with the framework and master
142   * @param tableName the name of the table
143   */
144  default void preDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
145      TableName tableName) throws IOException {}
146
147  /**
148   * Called after the deleteTable operation has been requested.  Called as part
149   * of delete table RPC call.
150   * @param ctx the environment to interact with the framework and master
151   * @param tableName the name of the table
152   */
153  default void postDeleteTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
154      TableName tableName) throws IOException {}
155
156  /**
157   * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
158   * table.  Called as part of delete table procedure and
159   * it is async to the delete RPC call.
160   *
161   * @param ctx the environment to interact with the framework and master
162   * @param tableName the name of the table
163   */
164  default void preDeleteTableAction(
165      final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
166      throws IOException {}
167
168  /**
169   * Called after {@link org.apache.hadoop.hbase.master.HMaster} deletes a
170   * table.  Called as part of delete table procedure and it is async to the
171   * delete RPC call.
172   *
173   * @param ctx the environment to interact with the framework and master
174   * @param tableName the name of the table
175   */
176  default void postCompletedDeleteTableAction(
177      final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
178      throws IOException {}
179
180  /**
181   * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
182   * table.  Called as part of truncate table RPC call.
183   * @param ctx the environment to interact with the framework and master
184   * @param tableName the name of the table
185   */
186  default void preTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
187      TableName tableName) throws IOException {}
188
189  /**
190   * Called after the truncateTable operation has been requested.  Called as part
191   * of truncate table RPC call.
192   * The truncate is synchronous, so this method will be called when the
193   * truncate operation is terminated.
194   * @param ctx the environment to interact with the framework and master
195   * @param tableName the name of the table
196   */
197  default void postTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
198      TableName tableName) throws IOException {}
199
200  /**
201   * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
202   * table.  Called as part of truncate table procedure and it is async
203   * to the truncate RPC call.
204   *
205   * @param ctx the environment to interact with the framework and master
206   * @param tableName the name of the table
207   */
208  default void preTruncateTableAction(
209      final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
210      throws IOException {}
211
212  /**
213   * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a
214   * table.  Called as part of truncate table procedure and it is async to the
215   * truncate RPC call.
216   *
217   * @param ctx the environment to interact with the framework and master
218   * @param tableName the name of the table
219   */
220  default void postCompletedTruncateTableAction(
221      final ObserverContext<MasterCoprocessorEnvironment> ctx, final TableName tableName)
222      throws IOException {}
223
224  /**
225   * Called prior to modifying a table's properties.  Called as part of modify
226   * table RPC call.
227   * @param ctx the environment to interact with the framework and master
228   * @param tableName the name of the table
229   * @param newDescriptor after modify operation, table will have this descriptor
230   * @deprecated Since 2.1. Will be removed in 3.0.
231   */
232  @Deprecated
233  default void preModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
234    final TableName tableName, TableDescriptor newDescriptor) throws IOException {}
235
236  /**
237   * Called prior to modifying a table's properties.  Called as part of modify
238   * table RPC call.
239   * @param ctx the environment to interact with the framework and master
240   * @param tableName the name of the table
241   * @param currentDescriptor current TableDescriptor of the table
242   * @param newDescriptor after modify operation, table will have this descriptor
243   */
244  default TableDescriptor preModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
245      final TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor)
246      throws IOException {
247    return newDescriptor;
248  }
249
250  /**
251   * Called after the modifyTable operation has been requested.  Called as part
252   * of modify table RPC call.
253   * @param ctx the environment to interact with the framework and master
254   * @param tableName the name of the table
255   * @param currentDescriptor current TableDescriptor of the table
256   * @deprecated Since 2.1. Will be removed in 3.0.
257   */
258  @Deprecated
259  default void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
260    final TableName tableName, TableDescriptor currentDescriptor) throws IOException {}
261
262  /**
263   * Called after the modifyTable operation has been requested.  Called as part
264   * of modify table RPC call.
265   * @param ctx the environment to interact with the framework and master
266   * @param tableName the name of the table
267   * @param oldDescriptor descriptor of table before modify operation happened
268   * @param currentDescriptor current TableDescriptor of the table
269   */
270  default void postModifyTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
271      final TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor)
272    throws IOException {
273    postModifyTable(ctx, tableName, currentDescriptor);
274  }
275
276  /**
277   * Called prior to modifying a table's properties.  Called as part of modify
278   * table procedure and it is async to the modify table RPC call.
279   *
280   * @param ctx the environment to interact with the framework and master
281   * @param tableName the name of the table
282   * @param newDescriptor after modify operation, table will have this descriptor
283   * @deprecated Since 2.1. Will be removed in 3.0.
284   */
285  @Deprecated
286  default void preModifyTableAction(
287    final ObserverContext<MasterCoprocessorEnvironment> ctx,
288    final TableName tableName,
289    final TableDescriptor newDescriptor) throws IOException {}
290
291  /**
292   * Called prior to modifying a table's properties.  Called as part of modify
293   * table procedure and it is async to the modify table RPC call.
294   *
295   * @param ctx the environment to interact with the framework and master
296   * @param tableName the name of the table
297   * @param currentDescriptor current TableDescriptor of the table
298   * @param newDescriptor after modify operation, table will have this descriptor
299   */
300  default void preModifyTableAction(
301      final ObserverContext<MasterCoprocessorEnvironment> ctx,
302      final TableName tableName,
303      final TableDescriptor currentDescriptor,
304      final TableDescriptor newDescriptor) throws IOException {
305    preModifyTableAction(ctx, tableName, newDescriptor);
306  }
307
308  /**
309   * Called after to modifying a table's properties.  Called as part of modify
310   * table procedure and it is async to the modify table RPC call.
311   *
312   * @param ctx the environment to interact with the framework and master
313   * @param tableName the name of the table
314   * @param currentDescriptor current TableDescriptor of the table
315   * @deprecated Since 2.1. Will be removed in 3.0.
316   */
317  @Deprecated
318  default void postCompletedModifyTableAction(
319    final ObserverContext<MasterCoprocessorEnvironment> ctx,
320    final TableName tableName,
321    final TableDescriptor currentDescriptor) throws IOException {}
322
323  /**
324   * Called after to modifying a table's properties.  Called as part of modify
325   * table procedure and it is async to the modify table RPC call.
326   *
327   * @param ctx the environment to interact with the framework and master
328   * @param tableName the name of the table
329   * @param oldDescriptor descriptor of table before modify operation happened
330   * @param currentDescriptor current TableDescriptor of the table
331   */
332  default void postCompletedModifyTableAction(
333      final ObserverContext<MasterCoprocessorEnvironment> ctx,
334      final TableName tableName,
335      final TableDescriptor oldDescriptor,
336      final TableDescriptor currentDescriptor) throws IOException {
337    postCompletedModifyTableAction(ctx, tableName, currentDescriptor);
338  }
339
340  /**
341   * Called prior to enabling a table.  Called as part of enable table RPC call.
342   * @param ctx the environment to interact with the framework and master
343   * @param tableName the name of the table
344   */
345  default void preEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
346      final TableName tableName) throws IOException {}
347
348  /**
349   * Called after the enableTable operation has been requested.  Called as part
350   * of enable table RPC call.
351   * @param ctx the environment to interact with the framework and master
352   * @param tableName the name of the table
353   */
354  default void postEnableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
355      final TableName tableName) throws IOException {}
356
357  /**
358   * Called prior to enabling a table.  Called as part of enable table procedure
359   * and it is async to the enable table RPC call.
360   *
361   * @param ctx the environment to interact with the framework and master
362   * @param tableName the name of the table
363   */
364  default void preEnableTableAction(
365      final ObserverContext<MasterCoprocessorEnvironment> ctx,
366      final TableName tableName) throws IOException {}
367
368  /**
369   * Called after the enableTable operation has been requested.  Called as part
370   * of enable table procedure and it is async to the enable table RPC call.
371   *
372   * @param ctx the environment to interact with the framework and master
373   * @param tableName the name of the table
374   */
375  default void postCompletedEnableTableAction(
376      final ObserverContext<MasterCoprocessorEnvironment> ctx,
377      final TableName tableName) throws IOException {}
378
379  /**
380   * Called prior to disabling a table.  Called as part of disable table RPC
381   * call.
382   * @param ctx the environment to interact with the framework and master
383   * @param tableName the name of the table
384   */
385  default void preDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
386      final TableName tableName) throws IOException {}
387
388  /**
389   * Called after the disableTable operation has been requested.  Called as part
390   * of disable table RPC call.
391   * @param ctx the environment to interact with the framework and master
392   * @param tableName the name of the table
393   */
394  default void postDisableTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
395      final TableName tableName) throws IOException {}
396
397  /**
398   * Called prior to disabling a table.  Called as part of disable table procedure
399   * and it is asyn to the disable table RPC call.
400   *
401   * @param ctx the environment to interact with the framework and master
402   * @param tableName the name of the table
403   */
404  default void preDisableTableAction(
405      final ObserverContext<MasterCoprocessorEnvironment> ctx,
406      final TableName tableName) throws IOException {}
407
408  /**
409   * Called after the disableTable operation has been requested.  Called as part
410   * of disable table procedure and it is asyn to the disable table RPC call.
411   *
412   * @param ctx the environment to interact with the framework and master
413   * @param tableName the name of the table
414   */
415  default void postCompletedDisableTableAction(
416      final ObserverContext<MasterCoprocessorEnvironment> ctx,
417      final TableName tableName) throws IOException {}
418
419  /**
420   * Called before a abortProcedure request has been processed.
421   * @param ctx the environment to interact with the framework and master
422   * @param procId the Id of the procedure
423   */
424  default void preAbortProcedure(
425      ObserverContext<MasterCoprocessorEnvironment> ctx, final long procId) throws IOException {}
426
427  /**
428   * Called after a abortProcedure request has been processed.
429   * @param ctx the environment to interact with the framework and master
430   */
431  default void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)
432      throws IOException {}
433
434  /**
435   * Called before a getProcedures request has been processed.
436   * @param ctx the environment to interact with the framework and master
437   */
438  default void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
439      throws IOException {}
440
441  /**
442   * Called after a getProcedures request has been processed.
443   * @param ctx the environment to interact with the framework and master
444   */
445  default void postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)
446      throws IOException {}
447
448  /**
449   * Called before a getLocks request has been processed.
450   * @param ctx the environment to interact with the framework and master
451   * @throws IOException if something went wrong
452   */
453  default void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)
454      throws IOException {}
455
456  /**
457   * Called after a getLocks request has been processed.
458   * @param ctx the environment to interact with the framework and master
459   * @throws IOException if something went wrong
460   */
461  default void postGetLocks(
462      ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException {}
463
464  /**
465   * Called prior to moving a given region from one region server to another.
466   * @param ctx the environment to interact with the framework and master
467   * @param region the RegionInfo
468   * @param srcServer the source ServerName
469   * @param destServer the destination ServerName
470   */
471  default void preMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
472      final RegionInfo region, final ServerName srcServer,
473      final ServerName destServer)
474    throws IOException {}
475
476  /**
477   * Called after the region move has been requested.
478   * @param ctx the environment to interact with the framework and master
479   * @param region the RegionInfo
480   * @param srcServer the source ServerName
481   * @param destServer the destination ServerName
482   */
483  default void postMove(final ObserverContext<MasterCoprocessorEnvironment> ctx,
484      final RegionInfo region, final ServerName srcServer,
485      final ServerName destServer)
486    throws IOException {}
487
488  /**
489   * Called prior to assigning a specific region.
490   * @param ctx the environment to interact with the framework and master
491   * @param regionInfo the regionInfo of the region
492   */
493  default void preAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
494      final RegionInfo regionInfo) throws IOException {}
495
496  /**
497   * Called after the region assignment has been requested.
498   * @param ctx the environment to interact with the framework and master
499   * @param regionInfo the regionInfo of the region
500   */
501  default void postAssign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
502      final RegionInfo regionInfo) throws IOException {}
503
504  /**
505   * Called prior to unassigning a given region.
506   * @param ctx the environment to interact with the framework and master
507   * @param regionInfo
508   * @param force whether to force unassignment or not
509   */
510  default void preUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
511      final RegionInfo regionInfo, final boolean force) throws IOException {}
512
513  /**
514   * Called after the region unassignment has been requested.
515   * @param ctx the environment to interact with the framework and master
516   * @param regionInfo
517   * @param force whether to force unassignment or not
518   */
519  default void postUnassign(final ObserverContext<MasterCoprocessorEnvironment> ctx,
520      final RegionInfo regionInfo, final boolean force) throws IOException {}
521
522  /**
523   * Called prior to marking a given region as offline.
524   * @param ctx the environment to interact with the framework and master
525   * @param regionInfo
526   */
527  default void preRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
528      final RegionInfo regionInfo) throws IOException {}
529
530  /**
531   * Called after the region has been marked offline.
532   * @param ctx the environment to interact with the framework and master
533   * @param regionInfo
534   */
535  default void postRegionOffline(final ObserverContext<MasterCoprocessorEnvironment> ctx,
536      final RegionInfo regionInfo) throws IOException {}
537
538  /**
539   * Called prior to requesting rebalancing of the cluster regions, though after
540   * the initial checks for regions in transition and the balance switch flag.
541   * @param ctx the environment to interact with the framework and master
542   */
543  default void preBalance(final ObserverContext<MasterCoprocessorEnvironment> ctx)
544      throws IOException {}
545
546  /**
547   * Called after the balancing plan has been submitted.
548   * @param ctx the environment to interact with the framework and master
549   * @param plans the RegionPlans which master has executed. RegionPlan serves as hint
550   * as for the final destination for the underlying region but may not represent the
551   * final state of assignment
552   */
553  default void postBalance(final ObserverContext<MasterCoprocessorEnvironment> ctx, List<RegionPlan> plans)
554      throws IOException {}
555
556  /**
557   * Called prior to setting split / merge switch
558   * Supports Coprocessor 'bypass'.
559   * @param ctx the coprocessor instance's environment
560   * @param newValue the new value submitted in the call
561   * @param switchType type of switch
562   */
563  default void preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
564      final boolean newValue, final MasterSwitchType switchType) throws IOException {}
565
566  /**
567   * Called after setting split / merge switch
568   * @param ctx the coprocessor instance's environment
569   * @param newValue the new value submitted in the call
570   * @param switchType type of switch
571   */
572  default void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
573      final boolean newValue, final MasterSwitchType switchType) throws IOException {}
574
575  /**
576   * Called before the split region procedure is called.
577   * @param c the environment to interact with the framework and master
578   * @param tableName the table where the region belongs to
579   * @param splitRow split point
580   */
581  default void preSplitRegion(
582      final ObserverContext<MasterCoprocessorEnvironment> c,
583      final TableName tableName,
584      final byte[] splitRow)
585      throws IOException {}
586
587  /**
588   * Called before the region is split.
589   * @param c the environment to interact with the framework and master
590   * @param tableName the table where the region belongs to
591   * @param splitRow split point
592   */
593  default void preSplitRegionAction(
594      final ObserverContext<MasterCoprocessorEnvironment> c,
595      final TableName tableName,
596      final byte[] splitRow)
597      throws IOException {}
598
599  /**
600   * Called after the region is split.
601   * @param c the environment to interact with the framework and master
602   * @param regionInfoA the left daughter region
603   * @param regionInfoB the right daughter region
604   */
605  default void postCompletedSplitRegionAction(
606      final ObserverContext<MasterCoprocessorEnvironment> c,
607      final RegionInfo regionInfoA,
608      final RegionInfo regionInfoB) throws IOException {}
609
610  /**
611   * This will be called before update META step as part of split transaction.
612   * @param ctx the environment to interact with the framework and master
613   * @param splitKey
614   * @param metaEntries
615   */
616  default void preSplitRegionBeforeMETAAction(
617      final ObserverContext<MasterCoprocessorEnvironment> ctx,
618      final byte[] splitKey,
619      final List<Mutation> metaEntries) throws IOException {}
620
621
622  /**
623   * This will be called after update META step as part of split transaction
624   * @param ctx the environment to interact with the framework and master
625   */
626  default void preSplitRegionAfterMETAAction(
627      final ObserverContext<MasterCoprocessorEnvironment> ctx)
628      throws IOException {}
629
630  /**
631   * This will be called after the roll back of the split region is completed
632   * @param ctx the environment to interact with the framework and master
633   */
634  default void postRollBackSplitRegionAction(
635      final ObserverContext<MasterCoprocessorEnvironment> ctx)
636      throws IOException {}
637
638  /**
639   * Called before the regions merge.
640   * @param ctx the environment to interact with the framework and master
641   */
642  default void preMergeRegionsAction(
643      final ObserverContext<MasterCoprocessorEnvironment> ctx,
644      final RegionInfo[] regionsToMerge) throws IOException {}
645
646  /**
647   * called after the regions merge.
648   * @param ctx the environment to interact with the framework and master
649   */
650  default void postCompletedMergeRegionsAction(
651      final ObserverContext<MasterCoprocessorEnvironment> ctx,
652      final RegionInfo[] regionsToMerge,
653      final RegionInfo mergedRegion) throws IOException {}
654
655  /**
656   * This will be called before update META step as part of regions merge transaction.
657   * @param ctx the environment to interact with the framework and master
658   * @param metaEntries mutations to execute on hbase:meta atomically with regions merge updates.
659   *        Any puts or deletes to execute on hbase:meta can be added to the mutations.
660   */
661  default void preMergeRegionsCommitAction(
662      final ObserverContext<MasterCoprocessorEnvironment> ctx,
663      final RegionInfo[] regionsToMerge,
664      @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException {}
665
666  /**
667   * This will be called after META step as part of regions merge transaction.
668   * @param ctx the environment to interact with the framework and master
669   */
670  default void postMergeRegionsCommitAction(
671      final ObserverContext<MasterCoprocessorEnvironment> ctx,
672      final RegionInfo[] regionsToMerge,
673      final RegionInfo mergedRegion) throws IOException {}
674
675  /**
676   * This will be called after the roll back of the regions merge.
677   * @param ctx the environment to interact with the framework and master
678   */
679  default void postRollBackMergeRegionsAction(
680      final ObserverContext<MasterCoprocessorEnvironment> ctx,
681      final RegionInfo[] regionsToMerge) throws IOException {}
682
683  /**
684   * Called prior to modifying the flag used to enable/disable region balancing.
685   * @param ctx the coprocessor instance's environment
686   */
687  default void preBalanceSwitch(final ObserverContext<MasterCoprocessorEnvironment> ctx,
688      final boolean newValue) throws IOException {}
689
690  /**
691   * Called after the flag to enable/disable balancing has changed.
692   * @param ctx the coprocessor instance's environment
693   * @param oldValue the previously set balanceSwitch value
694   * @param newValue the newly set balanceSwitch value
695   */
696  default void postBalanceSwitch(final ObserverContext<MasterCoprocessorEnvironment> ctx,
697      final boolean oldValue, final boolean newValue) throws IOException {}
698
699  /**
700   * Called prior to shutting down the full HBase cluster, including this
701   * {@link org.apache.hadoop.hbase.master.HMaster} process.
702   */
703  default void preShutdown(final ObserverContext<MasterCoprocessorEnvironment> ctx)
704      throws IOException {}
705
706
707  /**
708   * Called immediately prior to stopping this
709   * {@link org.apache.hadoop.hbase.master.HMaster} process.
710   */
711  default void preStopMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx)
712      throws IOException {}
713
714  /**
715   * Called immediately after an active master instance has completed
716   * initialization.  Will not be called on standby master instances unless
717   * they take over the active role.
718   */
719  default void postStartMaster(final ObserverContext<MasterCoprocessorEnvironment> ctx)
720      throws IOException {}
721
722  /**
723   * Call before the master initialization is set to true.
724   * {@link org.apache.hadoop.hbase.master.HMaster} process.
725   */
726  default void preMasterInitialization(final ObserverContext<MasterCoprocessorEnvironment> ctx)
727      throws IOException {}
728
729  /**
730   * Called before a new snapshot is taken.
731   * Called as part of snapshot RPC call.
732   * @param ctx the environment to interact with the framework and master
733   * @param snapshot the SnapshotDescriptor for the snapshot
734   * @param tableDescriptor the TableDescriptor of the table to snapshot
735   */
736  default void preSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
737      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
738      throws IOException {}
739
740  /**
741   * Called after the snapshot operation has been requested.
742   * Called as part of snapshot RPC call.
743   * @param ctx the environment to interact with the framework and master
744   * @param snapshot the SnapshotDescriptor for the snapshot
745   * @param tableDescriptor the TableDescriptor of the table to snapshot
746   */
747  default void postSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
748      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
749      throws IOException {}
750
751  /**
752   * Called after the snapshot operation has been completed.
753   * @param ctx the environment to interact with the framework and master
754   * @param snapshot the SnapshotDescriptor for the snapshot
755   * @param tableDescriptor the TableDescriptor of the table to snapshot
756   */
757  default void postCompletedSnapshotAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
758      SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException {
759  }
760
761  /**
762   * Called before listSnapshots request has been processed.
763   * @param ctx the environment to interact with the framework and master
764   * @param snapshot the SnapshotDescriptor of the snapshot to list
765   */
766  default void preListSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
767      final SnapshotDescription snapshot) throws IOException {}
768
769  /**
770   * Called after listSnapshots request has been processed.
771   * @param ctx the environment to interact with the framework and master
772   * @param snapshot the SnapshotDescriptor of the snapshot to list
773   */
774  default void postListSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
775      final SnapshotDescription snapshot) throws IOException {}
776
777  /**
778   * Called before a snapshot is cloned.
779   * Called as part of restoreSnapshot RPC call.
780   * @param ctx the environment to interact with the framework and master
781   * @param snapshot the SnapshotDescriptor for the snapshot
782   * @param tableDescriptor the TableDescriptor of the table to create
783   */
784  default void preCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
785      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
786      throws IOException {}
787
788  /**
789   * Called after a snapshot clone operation has been requested.
790   * Called as part of restoreSnapshot RPC call.
791   * @param ctx the environment to interact with the framework and master
792   * @param snapshot the SnapshotDescriptor for the snapshot
793   * @param tableDescriptor the v of the table to create
794   */
795  default void postCloneSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
796      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
797      throws IOException {}
798
799  /**
800   * Called before a snapshot is restored.
801   * Called as part of restoreSnapshot RPC call.
802   * @param ctx the environment to interact with the framework and master
803   * @param snapshot the SnapshotDescriptor for the snapshot
804   * @param tableDescriptor the TableDescriptor of the table to restore
805   */
806  default void preRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
807      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
808      throws IOException {}
809
810  /**
811   * Called after a snapshot restore operation has been requested.
812   * Called as part of restoreSnapshot RPC call.
813   * @param ctx the environment to interact with the framework and master
814   * @param snapshot the SnapshotDescriptor for the snapshot
815   * @param tableDescriptor the TableDescriptor of the table to restore
816   */
817  default void postRestoreSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
818      final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
819      throws IOException {}
820
821  /**
822   * Called before a snapshot is deleted.
823   * Called as part of deleteSnapshot RPC call.
824   * @param ctx the environment to interact with the framework and master
825   * @param snapshot the SnapshotDescriptor of the snapshot to delete
826   */
827  default void preDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
828      final SnapshotDescription snapshot) throws IOException {}
829
830  /**
831   * Called after the delete snapshot operation has been requested.
832   * Called as part of deleteSnapshot RPC call.
833   * @param ctx the environment to interact with the framework and master
834   * @param snapshot the SnapshotDescriptor of the snapshot to delete
835   */
836  default void postDeleteSnapshot(final ObserverContext<MasterCoprocessorEnvironment> ctx,
837      final SnapshotDescription snapshot) throws IOException {}
838
839  /**
840   * Called before a getTableDescriptors request has been processed.
841   * @param ctx the environment to interact with the framework and master
842   * @param tableNamesList the list of table names, or null if querying for all
843   * @param descriptors an empty list, can be filled with what to return in coprocessor
844   * @param regex regular expression used for filtering the table names
845   */
846  default void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
847      List<TableName> tableNamesList, List<TableDescriptor> descriptors,
848      String regex) throws IOException {}
849
850  /**
851   * Called after a getTableDescriptors request has been processed.
852   * @param ctx the environment to interact with the framework and master
853   * @param tableNamesList the list of table names, or null if querying for all
854   * @param descriptors the list of descriptors about to be returned
855   * @param regex regular expression used for filtering the table names
856   */
857  default void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
858      List<TableName> tableNamesList, List<TableDescriptor> descriptors,
859      String regex) throws IOException {}
860
861  /**
862   * Called before a getTableNames request has been processed.
863   * @param ctx the environment to interact with the framework and master
864   * @param descriptors an empty list, can be filled with what to return by coprocessor
865   * @param regex regular expression used for filtering the table names
866   */
867  default void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
868      List<TableDescriptor> descriptors, String regex) throws IOException {}
869
870  /**
871   * Called after a getTableNames request has been processed.
872   * @param ctx the environment to interact with the framework and master
873   * @param descriptors the list of descriptors about to be returned
874   * @param regex regular expression used for filtering the table names
875   */
876  default void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
877      List<TableDescriptor> descriptors, String regex) throws IOException {}
878
879
880
881  /**
882   * Called before a new namespace is created by
883   * {@link org.apache.hadoop.hbase.master.HMaster}.
884   * @param ctx the environment to interact with the framework and master
885   * @param ns the NamespaceDescriptor for the table
886   */
887  default void preCreateNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
888      NamespaceDescriptor ns) throws IOException {}
889  /**
890   * Called after the createNamespace operation has been requested.
891   * @param ctx the environment to interact with the framework and master
892   * @param ns the NamespaceDescriptor for the table
893   */
894  default void postCreateNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
895       NamespaceDescriptor ns) throws IOException {}
896
897  /**
898   * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a
899   * namespace
900   * @param ctx the environment to interact with the framework and master
901   * @param namespace the name of the namespace
902   */
903  default void preDeleteNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
904      String namespace) throws IOException {}
905
906  /**
907   * Called after the deleteNamespace operation has been requested.
908   * @param ctx the environment to interact with the framework and master
909   * @param namespace the name of the namespace
910   */
911  default void postDeleteNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
912      String namespace) throws IOException {}
913
914  /**
915   * Called prior to modifying a namespace's properties.
916   * @param ctx the environment to interact with the framework and master
917   * @param newNsDescriptor after modify operation, namespace will have this descriptor
918   * @deprecated Since 2.1. Will be removed in 3.0.
919   */
920  @Deprecated
921  default void preModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
922    NamespaceDescriptor newNsDescriptor) throws IOException {}
923
924  /**
925   * Called prior to modifying a namespace's properties.
926   * @param ctx the environment to interact with the framework and master
927   * @param currentNsDescriptor current NamespaceDescriptor of the namespace
928   * @param newNsDescriptor after modify operation, namespace will have this descriptor
929   */
930  default void preModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
931      NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor)
932    throws IOException {
933    preModifyNamespace(ctx, newNsDescriptor);
934  }
935
936  /**
937   * Called after the modifyNamespace operation has been requested.
938   * @param ctx the environment to interact with the framework and master
939   * @param currentNsDescriptor current NamespaceDescriptor of the namespace
940   * @deprecated Since 2.1. Will be removed in 3.0.
941   */
942  @Deprecated
943  default void postModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
944    NamespaceDescriptor currentNsDescriptor) throws IOException {}
945
946  /**
947   * Called after the modifyNamespace operation has been requested.
948   * @param ctx the environment to interact with the framework and master
949   * @param oldNsDescriptor descriptor of namespace before modify operation happened
950   * @param currentNsDescriptor current NamespaceDescriptor of the namespace
951   */
952  default void postModifyNamespace(final ObserverContext<MasterCoprocessorEnvironment> ctx,
953      NamespaceDescriptor oldNsDescriptor, NamespaceDescriptor currentNsDescriptor)
954    throws IOException {
955    postModifyNamespace(ctx, currentNsDescriptor);
956  }
957
958  /**
959   * Called before a getNamespaceDescriptor request has been processed.
960   * @param ctx the environment to interact with the framework and master
961   * @param namespace the name of the namespace
962   */
963  default void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
964      String namespace) throws IOException {}
965
966  /**
967   * Called after a getNamespaceDescriptor request has been processed.
968   * @param ctx the environment to interact with the framework and master
969   * @param ns the NamespaceDescriptor
970   */
971  default void postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
972      NamespaceDescriptor ns) throws IOException {}
973
974  /**
975   * Called before a listNamespaces request has been processed.
976   * @param ctx the environment to interact with the framework and master
977   * @param namespaces an empty list, can be filled with what to return if bypassing
978   * @throws IOException if something went wrong
979   */
980  default void preListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
981      List<String> namespaces) throws IOException {}
982
983  /**
984   * Called after a listNamespaces request has been processed.
985   * @param ctx the environment to interact with the framework and master
986   * @param namespaces the list of namespaces about to be returned
987   * @throws IOException if something went wrong
988   */
989  default void postListNamespaces(ObserverContext<MasterCoprocessorEnvironment> ctx,
990      List<String> namespaces)  throws IOException {};
991
992  /**
993   * Called before a listNamespaceDescriptors request has been processed.
994   * @param ctx the environment to interact with the framework and master
995   * @param descriptors an empty list, can be filled with what to return by coprocessor
996   */
997  default void preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
998      List<NamespaceDescriptor> descriptors) throws IOException {}
999
1000  /**
1001   * Called after a listNamespaceDescriptors request has been processed.
1002   * @param ctx the environment to interact with the framework and master
1003   * @param descriptors the list of descriptors about to be returned
1004   */
1005  default void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
1006      List<NamespaceDescriptor> descriptors) throws IOException {}
1007
1008
1009  /**
1010   * Called before the table memstore is flushed to disk.
1011   * @param ctx the environment to interact with the framework and master
1012   * @param tableName the name of the table
1013   */
1014  default void preTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1015      final TableName tableName) throws IOException {}
1016
1017  /**
1018   * Called after the table memstore is flushed to disk.
1019   * @param ctx the environment to interact with the framework and master
1020   * @param tableName the name of the table
1021   */
1022  default void postTableFlush(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1023      final TableName tableName) throws IOException {}
1024
1025  /**
1026   * Called before the quota for the user is stored.
1027   * @param ctx the environment to interact with the framework and master
1028   * @param userName the name of user
1029   * @param quotas the current quota for the user
1030   */
1031  default void preSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1032      final String userName, final GlobalQuotaSettings quotas) throws IOException {}
1033
1034  /**
1035   * Called after the quota for the user is stored.
1036   * @param ctx the environment to interact with the framework and master
1037   * @param userName the name of user
1038   * @param quotas the resulting quota for the user
1039   */
1040  default void postSetUserQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1041      final String userName, final GlobalQuotaSettings quotas) throws IOException {}
1042
1043  /**
1044   * Called before the quota for the user on the specified table is stored.
1045   * @param ctx the environment to interact with the framework and master
1046   * @param userName the name of user
1047   * @param tableName the name of the table
1048   * @param quotas the current quota for the user on the table
1049   */
1050  default void preSetUserQuota(
1051      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
1052      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
1053
1054  /**
1055   * Called after the quota for the user on the specified table is stored.
1056   * @param ctx the environment to interact with the framework and master
1057   * @param userName the name of user
1058   * @param tableName the name of the table
1059   * @param quotas the resulting quota for the user on the table
1060   */
1061  default void postSetUserQuota(
1062      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
1063      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
1064
1065  /**
1066   * Called before the quota for the user on the specified namespace is stored.
1067   * @param ctx the environment to interact with the framework and master
1068   * @param userName the name of user
1069   * @param namespace the name of the namespace
1070   * @param quotas the current quota for the user on the namespace
1071   */
1072  default void preSetUserQuota(
1073      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
1074      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
1075
1076  /**
1077   * Called after the quota for the user on the specified namespace is stored.
1078   * @param ctx the environment to interact with the framework and master
1079   * @param userName the name of user
1080   * @param namespace the name of the namespace
1081   * @param quotas the resulting quota for the user on the namespace
1082   */
1083  default void postSetUserQuota(
1084      final ObserverContext<MasterCoprocessorEnvironment> ctx, final String userName,
1085      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
1086
1087  /**
1088   * Called before the quota for the table is stored.
1089   * @param ctx the environment to interact with the framework and master
1090   * @param tableName the name of the table
1091   * @param quotas the current quota for the table
1092   */
1093  default void preSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1094      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
1095
1096  /**
1097   * Called after the quota for the table is stored.
1098   * @param ctx the environment to interact with the framework and master
1099   * @param tableName the name of the table
1100   * @param quotas the resulting quota for the table
1101   */
1102  default void postSetTableQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1103      final TableName tableName, final GlobalQuotaSettings quotas) throws IOException {}
1104
1105  /**
1106   * Called before the quota for the namespace is stored.
1107   * @param ctx the environment to interact with the framework and master
1108   * @param namespace the name of the namespace
1109   * @param quotas the current quota for the namespace
1110   */
1111  default void preSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1112      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
1113
1114  /**
1115   * Called after the quota for the namespace is stored.
1116   * @param ctx the environment to interact with the framework and master
1117   * @param namespace the name of the namespace
1118   * @param quotas the resulting quota for the namespace
1119   */
1120  default void postSetNamespaceQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1121      final String namespace, final GlobalQuotaSettings quotas) throws IOException {}
1122
1123  /**
1124   * Called before the quota for the region server is stored.
1125   * @param ctx the environment to interact with the framework and master
1126   * @param regionServer the name of the region server
1127   * @param quotas the current quota for the region server
1128   */
1129  default void preSetRegionServerQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1130      final String regionServer, final GlobalQuotaSettings quotas) throws IOException {}
1131
1132  /**
1133   * Called after the quota for the region server is stored.
1134   * @param ctx the environment to interact with the framework and master
1135   * @param regionServer the name of the region server
1136   * @param quotas the resulting quota for the region server
1137   */
1138  default void postSetRegionServerQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1139      final String regionServer, final GlobalQuotaSettings quotas) throws IOException {}
1140
1141  /**
1142   * Called before merge regions request.
1143   * @param ctx coprocessor environment
1144   * @param regionsToMerge regions to be merged
1145   */
1146  default void preMergeRegions(
1147      final ObserverContext<MasterCoprocessorEnvironment> ctx,
1148      final RegionInfo[] regionsToMerge) throws IOException {}
1149
1150  /**
1151   * called after merge regions request.
1152   * @param c coprocessor environment
1153   * @param regionsToMerge regions to be merged
1154   */
1155  default void postMergeRegions(
1156      final ObserverContext<MasterCoprocessorEnvironment> c,
1157      final RegionInfo[] regionsToMerge) throws IOException {}
1158
1159  /**
1160   * Called before servers are moved to target region server group
1161   * @param ctx the environment to interact with the framework and master
1162   * @param servers set of servers to move
1163   * @param targetGroup destination group
1164   */
1165  default void preMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1166                      Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {}
1167
1168  /**
1169   * Called after servers are moved to target region server group
1170   * @param ctx the environment to interact with the framework and master
1171   * @param servers set of servers to move
1172   * @param targetGroup name of group
1173   */
1174  default void postMoveServersAndTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1175      Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException {}
1176
1177  /**
1178   * Called before servers are moved to target region server group
1179   * @param ctx the environment to interact with the framework and master
1180   * @param servers set of servers to move
1181   * @param targetGroup destination group
1182   */
1183  default void preMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1184                      Set<Address> servers, String targetGroup) throws IOException {}
1185
1186  /**
1187   * Called after servers are moved to target region server group
1188   * @param ctx the environment to interact with the framework and master
1189   * @param servers set of servers to move
1190   * @param targetGroup name of group
1191   */
1192  default void postMoveServers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1193                       Set<Address> servers, String targetGroup) throws IOException {}
1194
1195  /**
1196   * Called before tables are moved to target region server group
1197   * @param ctx the environment to interact with the framework and master
1198   * @param tables set of tables to move
1199   * @param targetGroup name of group
1200   */
1201  default void preMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1202                     Set<TableName> tables, String targetGroup) throws IOException {}
1203
1204  /**
1205   * Called after servers are moved to target region server group
1206   * @param ctx the environment to interact with the framework and master
1207   * @param tables set of tables to move
1208   * @param targetGroup name of group
1209   */
1210  default void postMoveTables(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1211                      Set<TableName> tables, String targetGroup) throws IOException {}
1212
1213  /**
1214   * Called before a new region server group is added
1215   * @param ctx the environment to interact with the framework and master
1216   * @param name group name
1217   */
1218  default void preAddRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1219                     String name) throws IOException {}
1220
1221  /**
1222   * Called after a new region server group is added
1223   * @param ctx the environment to interact with the framework and master
1224   * @param name group name
1225   */
1226  default void postAddRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1227                      String name) throws IOException {}
1228
1229  /**
1230   * Called before a region server group is removed
1231   * @param ctx the environment to interact with the framework and master
1232   * @param name group name
1233   */
1234  default void preRemoveRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1235                        String name) throws IOException {}
1236
1237  /**
1238   * Called after a region server group is removed
1239   * @param ctx the environment to interact with the framework and master
1240   * @param name group name
1241   */
1242  default void postRemoveRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1243                         String name) throws IOException {}
1244
1245  /**
1246   * Called before a region server group is removed
1247   * @param ctx the environment to interact with the framework and master
1248   * @param groupName group name
1249   */
1250  default void preBalanceRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1251                         String groupName) throws IOException {}
1252
1253  /**
1254   * Called after a region server group is removed
1255   * @param ctx the environment to interact with the framework and master
1256   * @param groupName group name
1257   */
1258  default void postBalanceRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1259                          String groupName, boolean balancerRan) throws IOException {}
1260
1261  /**
1262   * Called before servers are removed from rsgroup
1263   * @param ctx the environment to interact with the framework and master
1264   * @param servers set of decommissioned servers to remove
1265   */
1266  default void preRemoveServers(
1267      final ObserverContext<MasterCoprocessorEnvironment> ctx,
1268      Set<Address> servers) throws IOException {}
1269
1270  /**
1271   * Called after servers are removed from rsgroup
1272   * @param ctx the environment to interact with the framework and master
1273   * @param servers set of servers to remove
1274   */
1275  default void postRemoveServers(
1276      final ObserverContext<MasterCoprocessorEnvironment> ctx,
1277      Set<Address> servers) throws IOException {}
1278
1279  /**
1280   * Called before add a replication peer
1281   * @param ctx the environment to interact with the framework and master
1282   * @param peerId a short name that identifies the peer
1283   * @param peerConfig configuration for the replication peer
1284   */
1285  default void preAddReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1286      String peerId, ReplicationPeerConfig peerConfig) throws IOException {}
1287
1288  /**
1289   * Called after add a replication peer
1290   * @param ctx the environment to interact with the framework and master
1291   * @param peerId a short name that identifies the peer
1292   * @param peerConfig configuration for the replication peer
1293   */
1294  default void postAddReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1295      String peerId, ReplicationPeerConfig peerConfig) throws IOException {}
1296
1297  /**
1298   * Called before remove a replication peer
1299   * @param ctx
1300   * @param peerId a short name that identifies the peer
1301   */
1302  default void preRemoveReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1303      String peerId) throws IOException {}
1304
1305  /**
1306   * Called after remove a replication peer
1307   * @param ctx
1308   * @param peerId a short name that identifies the peer
1309   */
1310  default void postRemoveReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1311      String peerId) throws IOException {}
1312
1313  /**
1314   * Called before enable a replication peer
1315   * @param ctx
1316   * @param peerId a short name that identifies the peer
1317   */
1318  default void preEnableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1319      String peerId) throws IOException {}
1320
1321  /**
1322   * Called after enable a replication peer
1323   * @param ctx
1324   * @param peerId a short name that identifies the peer
1325   */
1326  default void postEnableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1327      String peerId) throws IOException {}
1328
1329  /**
1330   * Called before disable a replication peer
1331   * @param ctx
1332   * @param peerId a short name that identifies the peer
1333   */
1334  default void preDisableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1335      String peerId) throws IOException {}
1336
1337  /**
1338   * Called after disable a replication peer
1339   * @param ctx
1340   * @param peerId a short name that identifies the peer
1341   */
1342  default void postDisableReplicationPeer(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1343      String peerId) throws IOException {}
1344
1345  /**
1346   * Called before get the configured ReplicationPeerConfig for the specified peer
1347   * @param ctx
1348   * @param peerId a short name that identifies the peer
1349   */
1350  default void preGetReplicationPeerConfig(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1351      String peerId) throws IOException {}
1352
1353  /**
1354   * Called after get the configured ReplicationPeerConfig for the specified peer
1355   * @param ctx
1356   * @param peerId a short name that identifies the peer
1357   */
1358  default void postGetReplicationPeerConfig(
1359      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException {}
1360
1361  /**
1362   * Called before update peerConfig for the specified peer
1363   * @param ctx
1364   * @param peerId a short name that identifies the peer
1365   */
1366  default void preUpdateReplicationPeerConfig(
1367      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
1368      ReplicationPeerConfig peerConfig) throws IOException {}
1369
1370  /**
1371   * Called after update peerConfig for the specified peer
1372   * @param ctx the environment to interact with the framework and master
1373   * @param peerId a short name that identifies the peer
1374   */
1375  default void postUpdateReplicationPeerConfig(
1376      final ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId,
1377      ReplicationPeerConfig peerConfig) throws IOException {}
1378
1379  /**
1380   * Called before list replication peers.
1381   * @param ctx the environment to interact with the framework and master
1382   * @param regex The regular expression to match peer id
1383   */
1384  default void preListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1385      String regex) throws IOException {}
1386
1387  /**
1388   * Called after list replication peers.
1389   * @param ctx the environment to interact with the framework and master
1390   * @param regex The regular expression to match peer id
1391   */
1392  default void postListReplicationPeers(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1393      String regex) throws IOException {}
1394
1395  /**
1396   * Called before new LockProcedure is queued.
1397   * @param ctx the environment to interact with the framework and master
1398   */
1399  default void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
1400      TableName tableName, RegionInfo[] regionInfos, String description) throws IOException {}
1401
1402  /**
1403   * Called after new LockProcedure is queued.
1404   * @param ctx the environment to interact with the framework and master
1405   */
1406  default void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace,
1407      TableName tableName, RegionInfo[] regionInfos, String description) throws IOException {}
1408
1409  /**
1410   * Called before heartbeat to a lock.
1411   * @param ctx the environment to interact with the framework and master
1412   */
1413  default void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx,
1414      TableName tn, String description) throws IOException {}
1415
1416  /**
1417   * Called after heartbeat to a lock.
1418   * @param ctx the environment to interact with the framework and master
1419   */
1420  default void postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx)
1421      throws IOException {}
1422
1423  /**
1424   * Called before get cluster status.
1425   */
1426  default void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)
1427      throws IOException {}
1428
1429  /**
1430   * Called after get cluster status.
1431   */
1432  default void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
1433    ClusterMetrics status) throws IOException {}
1434
1435  /**
1436   * Called before clear dead region servers.
1437   */
1438  default void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
1439      throws IOException {}
1440
1441  /**
1442   * Called after clear dead region servers.
1443   */
1444  default void postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
1445      List<ServerName> servers, List<ServerName> notClearedServers)
1446      throws IOException {}
1447
1448  /**
1449   * Called before decommission region servers.
1450   */
1451  default void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
1452      List<ServerName> servers, boolean offload) throws IOException {}
1453
1454  /**
1455   * Called after decommission region servers.
1456   */
1457  default void postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
1458      List<ServerName> servers, boolean offload) throws IOException {}
1459
1460  /**
1461   * Called before list decommissioned region servers.
1462   */
1463  default void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
1464      throws IOException {}
1465
1466  /**
1467   * Called after list decommissioned region servers.
1468   */
1469  default void postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)
1470      throws IOException {}
1471
1472  /**
1473   * Called before recommission region server.
1474   */
1475  default void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
1476      ServerName server, List<byte[]> encodedRegionNames) throws IOException {}
1477
1478  /**
1479   * Called after recommission region server.
1480   */
1481  default void postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
1482      ServerName server, List<byte[]> encodedRegionNames) throws IOException {}
1483
1484  /**
1485   * Called before switching rpc throttle enabled state.
1486   * @param ctx the coprocessor instance's environment
1487   * @param enable the rpc throttle value
1488   */
1489  default void preSwitchRpcThrottle(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1490      final boolean enable) throws IOException {
1491  }
1492
1493  /**
1494   * Called after switching rpc throttle enabled state.
1495   * @param ctx the coprocessor instance's environment
1496   * @param oldValue the previously rpc throttle value
1497   * @param newValue the newly rpc throttle value
1498   */
1499  default void postSwitchRpcThrottle(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1500      final boolean oldValue, final boolean newValue) throws IOException {
1501  }
1502
1503  /**
1504   * Called before getting if is rpc throttle enabled.
1505   * @param ctx the coprocessor instance's environment
1506   */
1507  default void preIsRpcThrottleEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx)
1508      throws IOException {
1509  }
1510
1511  /**
1512   * Called after getting if is rpc throttle enabled.
1513   * @param ctx the coprocessor instance's environment
1514   * @param rpcThrottleEnabled the rpc throttle enabled value
1515   */
1516  default void postIsRpcThrottleEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1517      final boolean rpcThrottleEnabled) throws IOException {
1518  }
1519
1520  /**
1521   * Called before switching exceed throttle quota state.
1522   * @param ctx the coprocessor instance's environment
1523   * @param enable the exceed throttle quota value
1524   */
1525  default void preSwitchExceedThrottleQuota(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1526      final boolean enable) throws IOException {
1527  }
1528
1529  /**
1530   * Called after switching exceed throttle quota state.
1531   * @param ctx the coprocessor instance's environment
1532   * @param oldValue the previously exceed throttle quota value
1533   * @param newValue the newly exceed throttle quota value
1534   */
1535  default void postSwitchExceedThrottleQuota(
1536      final ObserverContext<MasterCoprocessorEnvironment> ctx, final boolean oldValue,
1537      final boolean newValue) throws IOException {
1538  }
1539
1540  /**
1541   * Called before granting user permissions.
1542   * @param ctx the coprocessor instance's environment
1543   * @param userPermission the user and permissions
1544   * @param mergeExistingPermissions True if merge with previous granted permissions
1545   */
1546  default void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx,
1547      UserPermission userPermission, boolean mergeExistingPermissions) throws IOException {
1548  }
1549
1550  /**
1551   * Called after granting user permissions.
1552   * @param ctx the coprocessor instance's environment
1553   * @param userPermission the user and permissions
1554   * @param mergeExistingPermissions True if merge with previous granted permissions
1555   */
1556  default void postGrant(ObserverContext<MasterCoprocessorEnvironment> ctx,
1557      UserPermission userPermission, boolean mergeExistingPermissions) throws IOException {
1558  }
1559
1560  /**
1561   * Called before revoking user permissions.
1562   * @param ctx the coprocessor instance's environment
1563   * @param userPermission the user and permissions
1564   */
1565  default void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
1566      UserPermission userPermission) throws IOException {
1567  }
1568
1569  /**
1570   * Called after revoking user permissions.
1571   * @param ctx the coprocessor instance's environment
1572   * @param userPermission the user and permissions
1573   */
1574  default void postRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
1575      UserPermission userPermission) throws IOException {
1576  }
1577
1578  /**
1579   * Called before getting user permissions.
1580   * @param ctx the coprocessor instance's environment
1581   * @param userName the user name, null if get all user permissions
1582   * @param namespace the namespace, null if don't get namespace permission
1583   * @param tableName the table name, null if don't get table permission
1584   * @param family the table column family, null if don't get table family permission
1585   * @param qualifier the table column qualifier, null if don't get table qualifier permission
1586   * @throws IOException if something went wrong
1587   */
1588  default void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
1589      String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
1590      throws IOException {
1591  }
1592
1593  /**
1594   * Called after getting user permissions.
1595   * @param ctx the coprocessor instance's environment
1596   * @param userName the user name, null if get all user permissions
1597   * @param namespace the namespace, null if don't get namespace permission
1598   * @param tableName the table name, null if don't get table permission
1599   * @param family the table column family, null if don't get table family permission
1600   * @param qualifier the table column qualifier, null if don't get table qualifier permission
1601   * @throws IOException if something went wrong
1602   */
1603  default void postGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
1604      String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier)
1605      throws IOException {
1606  }
1607
1608  /*
1609   * Called before checking if user has permissions.
1610   * @param ctx the coprocessor instance's environment
1611   * @param userName the user name
1612   * @param permissions the permission list
1613   */
1614  default void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
1615      String userName, List<Permission> permissions) throws IOException {
1616  }
1617
1618  /**
1619   * Called after checking if user has permissions.
1620   * @param ctx the coprocessor instance's environment
1621   * @param userName the user name
1622   * @param permissions the permission list
1623   */
1624  default void postHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
1625      String userName, List<Permission> permissions) throws IOException {
1626  }
1627
1628  /**
1629   * Called before rename rsgroup.
1630   * @param ctx the environment to interact with the framework and master
1631   * @param oldName old rsgroup name
1632   * @param newName new rsgroup name
1633   * @throws IOException on failure
1634   */
1635  default void preRenameRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1636      final String oldName, final String newName) throws IOException {
1637  }
1638
1639  /**
1640   * Called after rename rsgroup.
1641   * @param ctx the environment to interact with the framework and master
1642   * @param oldName old rsgroup name
1643   * @param newName new rsgroup name
1644   * @throws IOException on failure
1645   */
1646  default void postRenameRSGroup(final ObserverContext<MasterCoprocessorEnvironment> ctx,
1647      final String oldName, final String newName) throws IOException {
1648  }
1649}