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