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