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