@InterfaceAudience.LimitedPrivate(value="Coprocesssor") @InterfaceStability.Evolving public interface MasterObserver
HMaster process.
 IOException are reported back to client.CoprocessorHost.ABORT_ON_ERROR_KEY is set to true, then
         the server aborts.DoNotRetryIOException is returned to the client.| Modifier and Type | Method and Description | 
|---|---|
| default void | postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx)Called after a abortProcedure request has been processed. | 
| default void | postAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                      String peerId,
                      ReplicationPeerConfig peerConfig)Called after add a replication peer | 
| default void | postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
              String name)Called after a new region server group is added | 
| default void | postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
          RegionInfo regionInfo)Called after the region assignment has been requested. | 
| default void | postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx,
           List<RegionPlan> plans)Called after the balancing plan has been submitted. | 
| default void | postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  String groupName,
                  boolean balancerRan)Called after a region server group is removed | 
| default void | postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 boolean oldValue,
                 boolean newValue)Called after the flag to enable/disable balancing has changed. | 
| default void | postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    List<ServerName> servers,
                    List<ServerName> notClearedServers)Called after clear dead region servers. | 
| default void | postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 SnapshotDescription snapshot,
                 TableDescriptor tableDescriptor)Called after a snapshot clone operation has been requested. | 
| default void | postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              TableDescriptor desc,
                              RegionInfo[] regions)Called after the createTable operation has been requested. | 
| default void | postCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              TableName tableName)Called after  HMasterdeletes a
 table. | 
| default void | postCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                               TableName tableName)Called after the disableTable operation has been requested. | 
| default void | postCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              TableName tableName)Called after the enableTable operation has been requested. | 
| default void | postCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                               RegionInfo[] regionsToMerge,
                               RegionInfo mergedRegion)called after the regions merge. | 
| default void | postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              TableName tableName,
                              TableDescriptor currentDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default void | postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              TableName tableName,
                              TableDescriptor oldDescriptor,
                              TableDescriptor currentDescriptor)Called after to modifying a table's properties. | 
| default void | postCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c,
                              RegionInfo regionInfoA,
                              RegionInfo regionInfoB)Called after the region is split. | 
| default void | postCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                                TableName tableName)Called after  HMastertruncates a
 table. | 
| default void | postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                   NamespaceDescriptor ns)Called after the createNamespace operation has been requested. | 
| default void | postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableDescriptor desc,
               RegionInfo[] regions)Called after the createTable operation has been requested. | 
| default void | postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                             List<ServerName> servers,
                             boolean offload)Called after decommission region servers. | 
| default void | postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                   String namespace)Called after the deleteNamespace operation has been requested. | 
| default void | postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  SnapshotDescription snapshot)Called after the delete snapshot operation has been requested. | 
| default void | postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableName tableName)Called after the deleteTable operation has been requested. | 
| default void | postDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                          String peerId)Called after disable a replication peer | 
| default void | postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
                TableName tableName)Called after the disableTable operation has been requested. | 
| default void | postEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                         String peerId)Called after enable a replication peer | 
| default void | postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableName tableName)Called after the enableTable operation has been requested. | 
| default void | postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     ClusterMetrics status)Called after get cluster status. | 
| default void | postGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)Called after a getLocks request has been processed. | 
| default void | postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
                          NamespaceDescriptor ns)Called after a getNamespaceDescriptor request has been processed. | 
| default void | postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)Called after a getProcedures request has been processed. | 
| default void | postGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            String peerId)Called after get the configured ReplicationPeerConfig for the specified peer | 
| default void | postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
                       List<TableName> tableNamesList,
                       List<TableDescriptor> descriptors,
                       String regex)Called after a getTableDescriptors request has been processed. | 
| default void | postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 List<TableDescriptor> descriptors,
                 String regex)Called after a getTableNames request has been processed. | 
| default void | postGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
                      String userName,
                      String namespace,
                      TableName tableName,
                      byte[] family,
                      byte[] qualifier)Called after getting user permissions. | 
| default void | postGrant(ObserverContext<MasterCoprocessorEnvironment> ctx,
         UserPermission userPermission,
         boolean mergeExistingPermissions)Called after granting user permissions. | 
| default void | postHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
                      String userName,
                      List<Permission> permissions)Called after checking if user has permissions. | 
| default void | postIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        boolean rpcThrottleEnabled)Called after getting if is rpc throttle enabled. | 
| default void | postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)Called after list decommissioned region servers. | 
| default void | postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            List<NamespaceDescriptor> descriptors)Called after a listNamespaceDescriptors request has been processed. | 
| default void | postListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        String regex)Called after list replication peers. | 
| default void | postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                SnapshotDescription snapshot)Called after listSnapshots request has been processed. | 
| default void | postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx)Called after heartbeat to a lock. | 
| default void | postMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c,
                RegionInfo[] regionsToMerge)called after merge regions request. | 
| default void | postMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            RegionInfo[] regionsToMerge,
                            RegionInfo mergedRegion)This will be called after META step as part of regions merge transaction. | 
| default void | postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                   NamespaceDescriptor currentNsDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default void | postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                   NamespaceDescriptor oldNsDescriptor,
                   NamespaceDescriptor currentNsDescriptor)Called after the modifyNamespace operation has been requested. | 
| default void | postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableName tableName,
               TableDescriptor currentDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default void | postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableName tableName,
               TableDescriptor oldDescriptor,
               TableDescriptor currentDescriptor)Called after the modifyTable operation has been requested. | 
| default void | postMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
        RegionInfo region,
        ServerName srcServer,
        ServerName destServer)Called after the region move has been requested. | 
| default void | postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
               Set<Address> servers,
               String targetGroup)Called after servers are moved to target region server group | 
| default void | postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        Set<Address> servers,
                        Set<TableName> tables,
                        String targetGroup)Called after servers are moved to target region server group | 
| default void | postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
              Set<TableName> tables,
              String targetGroup)Called after servers are moved to target region server group | 
| default void | postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            ServerName server,
                            List<byte[]> encodedRegionNames)Called after recommission region server. | 
| default void | postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 RegionInfo regionInfo)Called after the region has been marked offline. | 
| default void | postRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                         String peerId)Called after remove a replication peer | 
| default void | postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 String name)Called after a region server group is removed | 
| default void | postRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 Set<Address> servers)Called after servers are removed from rsgroup | 
| default void | postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx,
               String namespace,
               TableName tableName,
               RegionInfo[] regionInfos,
               String description)Called after new LockProcedure is queued. | 
| default void | postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                   SnapshotDescription snapshot,
                   TableDescriptor tableDescriptor)Called after a snapshot restore operation has been requested. | 
| default void | postRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
          UserPermission userPermission)Called after revoking user permissions. | 
| default void | postRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              RegionInfo[] regionsToMerge)This will be called after the roll back of the regions merge. | 
| default void | postRollBackSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> ctx)This will be called after the roll back of the split region is completed | 
| default void | postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     String namespace,
                     GlobalQuotaSettings quotas)Called after the quota for the namespace is stored. | 
| default void | postSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        String regionServer,
                        GlobalQuotaSettings quotas)Called after the quota for the region server is stored. | 
| default void | postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
                          boolean newValue,
                          MasterSwitchType switchType)Called after setting split / merge switch | 
| default void | postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 TableName tableName,
                 GlobalQuotaSettings quotas)Called after the quota for the table is stored. | 
| default void | postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                String userName,
                GlobalQuotaSettings quotas)Called after the quota for the user is stored. | 
| default void | postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                String userName,
                String namespace,
                GlobalQuotaSettings quotas)Called after the quota for the user on the specified namespace is stored. | 
| default void | postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                String userName,
                TableName tableName,
                GlobalQuotaSettings quotas)Called after the quota for the user on the specified table is stored. | 
| default void | postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
            SnapshotDescription snapshot,
            TableDescriptor tableDescriptor)Called after the snapshot operation has been requested. | 
| default void | postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)Called immediately after an active master instance has completed
 initialization. | 
| default void | postSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                             boolean oldValue,
                             boolean newValue)Called after switching exceed throttle quota state. | 
| default void | postSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     boolean oldValue,
                     boolean newValue)Called after switching rpc throttle enabled state. | 
| default void | postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableName tableName)Called after the table memstore is flushed to disk. | 
| default void | postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 TableName tableName)Called after the truncateTable operation has been requested. | 
| default void | postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
            RegionInfo regionInfo,
            boolean force)Called after the region unassignment has been requested. | 
| default void | postUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
                               String peerId,
                               ReplicationPeerConfig peerConfig)Called after update peerConfig for the specified peer | 
| default void | preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 long procId)Called before a abortProcedure request has been processed. | 
| default void | preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     String peerId,
                     ReplicationPeerConfig peerConfig)Called before add a replication peer | 
| default void | preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
             String name)Called before a new region server group is added | 
| default void | preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx,
         RegionInfo regionInfo)Called prior to assigning a specific region. | 
| default void | preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx)Called prior to requesting rebalancing of the cluster regions, though after
 the initial checks for regions in transition and the balance switch flag. | 
| default void | preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 String groupName)Called before a region server group is removed | 
| default void | preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx,
                boolean newValue)Called prior to modifying the flag used to enable/disable region balancing. | 
| default void | preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before clear dead region servers. | 
| default void | preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                SnapshotDescription snapshot,
                TableDescriptor tableDescriptor)Called before a snapshot is cloned. | 
| default void | preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  NamespaceDescriptor ns)Called before a new namespace is created by
  HMaster. | 
| default void | preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableDescriptor desc,
              RegionInfo[] regions)Called before a new table is created by
  HMaster. | 
| default void | preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    TableDescriptor desc,
                    RegionInfo[] regions)Called before a new table is created by
  HMaster. | 
| default TableDescriptor | preCreateTableRegionsInfos(ObserverContext<MasterCoprocessorEnvironment> ctx,
                          TableDescriptor desc)Called before we create the region infos for this table. | 
| default void | preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            List<ServerName> servers,
                            boolean offload)Called before decommission region servers. | 
| default void | preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  String namespace)Called before  HMasterdeletes a
 namespace | 
| default void | preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                 SnapshotDescription snapshot)Called before a snapshot is deleted. | 
| default void | preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableName tableName)Called before  HMasterdeletes a
 table. | 
| default void | preDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    TableName tableName)Called before  HMasterdeletes a
 table. | 
| default void | preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                         String peerId)Called before disable a replication peer | 
| default void | preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
               TableName tableName)Called prior to disabling a table. | 
| default void | preDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     TableName tableName)Called prior to disabling a table. | 
| default void | preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        String peerId)Called before enable a replication peer | 
| default void | preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableName tableName)Called prior to enabling a table. | 
| default void | preEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    TableName tableName)Called prior to enabling a table. | 
| default void | preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before get cluster status. | 
| default void | preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before a getLocks request has been processed. | 
| default void | preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx,
                         String namespace)Called before a getNamespaceDescriptor request has been processed. | 
| default void | preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before a getProcedures request has been processed. | 
| default void | preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
                           String peerId)Called before get the configured ReplicationPeerConfig for the specified peer | 
| default void | preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
                      List<TableName> tableNamesList,
                      List<TableDescriptor> descriptors,
                      String regex)Called before a getTableDescriptors request has been processed. | 
| default void | preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx,
                List<TableDescriptor> descriptors,
                String regex)Called before a getTableNames request has been processed. | 
| default void | preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     String userName,
                     String namespace,
                     TableName tableName,
                     byte[] family,
                     byte[] qualifier)Called before getting user permissions. | 
| default void | preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx,
        UserPermission userPermission,
        boolean mergeExistingPermissions)Called before granting user permissions. | 
| default void | preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     String userName,
                     List<Permission> permissions) | 
| default void | preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before getting if is rpc throttle enabled. | 
| default void | preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx)Called before list decommissioned region servers. | 
| default void | preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx,
                           List<NamespaceDescriptor> descriptors)Called before a listNamespaceDescriptors request has been processed. | 
| default void | preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                       String regex)Called before list replication peers. | 
| default void | preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
               SnapshotDescription snapshot)Called before listSnapshots request has been processed. | 
| default void | preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx,
                TableName tn,
                String description)Called before heartbeat to a lock. | 
| default void | preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx)Call before the master initialization is set to true. | 
| default void | preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx,
               RegionInfo[] regionsToMerge)Called before merge regions request. | 
| default void | preMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                     RegionInfo[] regionsToMerge)Called before the regions merge. | 
| default void | preMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                           RegionInfo[] regionsToMerge,
                           List<Mutation> metaEntries)This will be called before update META step as part of regions merge transaction. | 
| default void | preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  NamespaceDescriptor newNsDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default void | preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  NamespaceDescriptor currentNsDescriptor,
                  NamespaceDescriptor newNsDescriptor)Called prior to modifying a namespace's properties. | 
| default void | preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableName tableName,
              TableDescriptor newDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default TableDescriptor | preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
              TableName tableName,
              TableDescriptor currentDescriptor,
              TableDescriptor newDescriptor)Called prior to modifying a table's properties. | 
| default void | preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    TableName tableName,
                    TableDescriptor newDescriptor)Deprecated. 
 Since 2.1. Will be removed in 3.0. | 
| default void | preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    TableName tableName,
                    TableDescriptor currentDescriptor,
                    TableDescriptor newDescriptor)Called prior to modifying a table's properties. | 
| default void | preMove(ObserverContext<MasterCoprocessorEnvironment> ctx,
       RegionInfo region,
       ServerName srcServer,
       ServerName destServer)Called prior to moving a given region from one region server to another. | 
| default void | preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
              Set<Address> servers,
              String targetGroup)Called before servers are moved to target region server group | 
| default void | preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
                       Set<Address> servers,
                       Set<TableName> tables,
                       String targetGroup)Called before servers are moved to target region server group | 
| default void | preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx,
             Set<TableName> tables,
             String targetGroup)Called before tables are moved to target region server group | 
| default void | preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                           ServerName server,
                           List<byte[]> encodedRegionNames)Called before recommission region server. | 
| default void | preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx,
                RegionInfo regionInfo)Called prior to marking a given region as offline. | 
| default void | preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx,
                        String peerId)Called before remove a replication peer | 
| default void | preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx,
                String name)Called before a region server group is removed | 
| default void | preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx,
                Set<Address> servers)Called before servers are removed from rsgroup | 
| default void | preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx,
              String namespace,
              TableName tableName,
              RegionInfo[] regionInfos,
              String description)Called before new LockProcedure is queued. | 
| default void | preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
                  SnapshotDescription snapshot,
                  TableDescriptor tableDescriptor)Called before a snapshot is restored. | 
| default void | preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx,
         UserPermission userPermission)Called before revoking user permissions. | 
| default void | preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    String namespace,
                    GlobalQuotaSettings quotas)Called before the quota for the namespace is stored. | 
| default void | preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                       String regionServer,
                       GlobalQuotaSettings quotas)Called before the quota for the region server is stored. | 
| default void | preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
                         boolean newValue,
                         MasterSwitchType switchType)Called prior to setting split / merge switch
 Supports Coprocessor 'bypass'. | 
| default void | preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                TableName tableName,
                GlobalQuotaSettings quotas)Called before the quota for the table is stored. | 
| default void | preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
               String userName,
               GlobalQuotaSettings quotas)Called before the quota for the user is stored. | 
| default void | preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
               String userName,
               String namespace,
               GlobalQuotaSettings quotas)Called before the quota for the user on the specified namespace is stored. | 
| default void | preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
               String userName,
               TableName tableName,
               GlobalQuotaSettings quotas)Called before the quota for the user on the specified table is stored. | 
| default void | preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx)Called prior to shutting down the full HBase cluster, including this
  HMasterprocess. | 
| default void | preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx,
           SnapshotDescription snapshot,
           TableDescriptor tableDescriptor)Called before a new snapshot is taken. | 
| default void | preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c,
              TableName tableName,
              byte[] splitRow)Called before the split region procedure is called. | 
| default void | preSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c,
                    TableName tableName,
                    byte[] splitRow)Called before the region is split. | 
| default void | preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx)This will be called after update META step as part of split transaction | 
| default void | preSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              byte[] splitKey,
                              List<Mutation> metaEntries)This will be called before update META step as part of split transaction. | 
| default void | preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)Called immediately prior to stopping this
  HMasterprocess. | 
| default void | preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx,
                            boolean enable)Called before switching exceed throttle quota state. | 
| default void | preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx,
                    boolean enable)Called before switching rpc throttle enabled state. | 
| default void | preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx,
             TableName tableName)Called before the table memstore is flushed to disk. | 
| default void | preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
                TableName tableName)Called before  HMastertruncates a
 table. | 
| default void | preTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx,
                      TableName tableName)Called before  HMastertruncates a
 table. | 
| default void | preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx,
           RegionInfo regionInfo,
           boolean force)Called prior to unassigning a given region. | 
| default void | preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx,
                              String peerId,
                              ReplicationPeerConfig peerConfig)Called before update peerConfig for the specified peer | 
default TableDescriptor preCreateTableRegionsInfos(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc) throws IOException
ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tablenull means cancel the creation.IOExceptiondefault void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
HMaster.  Called as part of create
 table RPC call.ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void preCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
HMaster.  Called as part of create
 table procedure and it is async to the create RPC call.ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void postCompletedCreateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableDescriptor desc, RegionInfo[] regions) throws IOException
ctx - the environment to interact with the framework and masterdesc - the TableDescriptor for the tableregions - the initial regions created for the tableIOExceptiondefault void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
 table.  Called as part of delete table RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
 table.  Called as part of delete table procedure and
 it is async to the delete RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedDeleteTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster deletes a
 table.  Called as part of delete table procedure and it is async to the
 delete RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
 table.  Called as part of truncate table RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
 table.  Called as part of truncate table procedure and it is async
 to the truncate RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedTruncateTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
HMaster truncates a
 table.  Called as part of truncate table procedure and it is async to the
 truncate RPC call.ctx - the environment to interact with the framework and mastertableName - the name of the tableIOException@Deprecated default void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablenewDescriptor - after modify operation, table will have this descriptorIOExceptiondefault TableDescriptor preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tablenewDescriptor - after modify operation, table will have this descriptorIOException@Deprecated default void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tableIOExceptiondefault void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableoldDescriptor - descriptor of table before modify operation happenedcurrentDescriptor - current TableDescriptor of the tableIOException@Deprecated default void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablenewDescriptor - after modify operation, table will have this descriptorIOExceptiondefault void preModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor, TableDescriptor newDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tablenewDescriptor - after modify operation, table will have this descriptorIOException@Deprecated default void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablecurrentDescriptor - current TableDescriptor of the tableIOExceptiondefault void postCompletedModifyTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, TableDescriptor oldDescriptor, TableDescriptor currentDescriptor) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableoldDescriptor - descriptor of table before modify operation happenedcurrentDescriptor - current TableDescriptor of the tableIOExceptiondefault void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedEnableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postCompletedDisableTableAction(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx, long procId) throws IOException
ctx - the environment to interact with the framework and masterprocId - the Id of the procedureIOExceptiondefault void postAbortProcedure(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postGetProcedures(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOException - if something went wrongdefault void postGetLocks(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOException - if something went wrongdefault void preMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the RegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptiondefault void postMove(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo region, ServerName srcServer, ServerName destServer) throws IOException
ctx - the environment to interact with the framework and masterregion - the RegionInfosrcServer - the source ServerNamedestServer - the destination ServerNameIOExceptiondefault void preAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptiondefault void postAssign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - the regionInfo of the regionIOExceptiondefault void preUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptiondefault void postUnassign(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo, boolean force) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - force - whether to force unassignment or notIOExceptiondefault void preRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - IOExceptiondefault void postRegionOffline(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo regionInfo) throws IOException
ctx - the environment to interact with the framework and masterregionInfo - IOExceptiondefault void preBalance(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postBalance(ObserverContext<MasterCoprocessorEnvironment> ctx, List<RegionPlan> plans) throws IOException
ctx - the environment to interact with the framework and masterplans - the RegionPlans which master has executed. RegionPlan serves as hint
 as for the final destination for the underlying region but may not represent the
 final state of assignmentIOExceptiondefault void preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
ctx - the coprocessor instance's environmentnewValue - the new value submitted in the callswitchType - type of switchIOExceptiondefault void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue, MasterSwitchType switchType) throws IOException
ctx - the coprocessor instance's environmentnewValue - the new value submitted in the callswitchType - type of switchIOExceptiondefault void preSplitRegion(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException
c - the environment to interact with the framework and mastertableName - the table where the region belongs tosplitRow - split pointIOExceptiondefault void preSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName, byte[] splitRow) throws IOException
c - the environment to interact with the framework and mastertableName - the table where the region belongs tosplitRow - split pointIOExceptiondefault void postCompletedSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo regionInfoA, RegionInfo regionInfoB) throws IOException
c - the environment to interact with the framework and masterregionInfoA - the left daughter regionregionInfoB - the right daughter regionIOExceptiondefault void preSplitRegionBeforeMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx, byte[] splitKey, List<Mutation> metaEntries) throws IOException
ctx - the environment to interact with the framework and mastersplitKey - metaEntries - IOExceptiondefault void preSplitRegionAfterMETAAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRollBackSplitRegionAction(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postCompletedMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, @MetaMutationAnnotation List<Mutation> metaEntries) throws IOException
ctx - the environment to interact with the framework and mastermetaEntries - mutations to execute on hbase:meta atomically with regions merge updates.
        Any puts or deletes to execute on hbase:meta can be added to the mutations.IOExceptiondefault void postMergeRegionsCommitAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge, RegionInfo mergedRegion) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRollBackMergeRegionsAction(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentIOExceptiondefault void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentoldValue - the previously set balanceSwitch valuenewValue - the newly set balanceSwitch valueIOExceptiondefault void preShutdown(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void preMasterInitialization(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
HMaster process.IOExceptiondefault void preSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to snapshotIOExceptiondefault void postSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to snapshotIOExceptiondefault void preListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptiondefault void postListSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to listIOExceptiondefault void preCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to createIOExceptiondefault void postCloneSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the v of the table to createIOExceptiondefault void preRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to restoreIOExceptiondefault void postRestoreSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot, TableDescriptor tableDescriptor) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor for the snapshottableDescriptor - the TableDescriptor of the table to restoreIOExceptiondefault void preDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOExceptiondefault void postDeleteSnapshot(ObserverContext<MasterCoprocessorEnvironment> ctx, SnapshotDescription snapshot) throws IOException
ctx - the environment to interact with the framework and mastersnapshot - the SnapshotDescriptor of the snapshot to deleteIOExceptiondefault void preGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
ctx - the environment to interact with the framework and mastertableNamesList - the list of table names, or null if querying for alldescriptors - an empty list, can be filled with what to return in coprocessorregex - regular expression used for filtering the table namesIOExceptiondefault void postGetTableDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableName> tableNamesList, List<TableDescriptor> descriptors, String regex) throws IOException
ctx - the environment to interact with the framework and mastertableNamesList - the list of table names, or null if querying for alldescriptors - the list of descriptors about to be returnedregex - regular expression used for filtering the table namesIOExceptiondefault void preGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - an empty list, can be filled with what to return by coprocessorregex - regular expression used for filtering the table namesIOExceptiondefault void postGetTableNames(ObserverContext<MasterCoprocessorEnvironment> ctx, List<TableDescriptor> descriptors, String regex) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - the list of descriptors about to be returnedregex - regular expression used for filtering the table namesIOExceptiondefault void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
HMaster.ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptiondefault void postCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptor for the tableIOExceptiondefault void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
HMaster deletes a
 namespacectx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptiondefault void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOException@Deprecated default void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor newNsDescriptor) throws IOException
ctx - the environment to interact with the framework and masternewNsDescriptor - after modify operation, namespace will have this descriptorIOExceptiondefault void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor, NamespaceDescriptor newNsDescriptor) throws IOException
ctx - the environment to interact with the framework and mastercurrentNsDescriptor - current NamespaceDescriptor of the namespacenewNsDescriptor - after modify operation, namespace will have this descriptorIOException@Deprecated default void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor currentNsDescriptor) throws IOException
ctx - the environment to interact with the framework and mastercurrentNsDescriptor - current NamespaceDescriptor of the namespaceIOExceptiondefault void postModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor oldNsDescriptor, NamespaceDescriptor currentNsDescriptor) throws IOException
ctx - the environment to interact with the framework and masteroldNsDescriptor - descriptor of namespace before modify operation happenedcurrentNsDescriptor - current NamespaceDescriptor of the namespaceIOExceptiondefault void preGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespaceIOExceptiondefault void postGetNamespaceDescriptor(ObserverContext<MasterCoprocessorEnvironment> ctx, NamespaceDescriptor ns) throws IOException
ctx - the environment to interact with the framework and masterns - the NamespaceDescriptorIOExceptiondefault void preListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - an empty list, can be filled with what to return by coprocessorIOExceptiondefault void postListNamespaceDescriptors(ObserverContext<MasterCoprocessorEnvironment> ctx, List<NamespaceDescriptor> descriptors) throws IOException
ctx - the environment to interact with the framework and masterdescriptors - the list of descriptors about to be returnedIOExceptiondefault void preTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void postTableFlush(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tableIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the current quota for the userIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of userquotas - the resulting quota for the userIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usertableName - the name of the tablequotas - the current quota for the user on the tableIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usertableName - the name of the tablequotas - the resulting quota for the user on the tableIOExceptiondefault void preSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usernamespace - the name of the namespacequotas - the current quota for the user on the namespaceIOExceptiondefault void postSetUserQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masteruserName - the name of usernamespace - the name of the namespacequotas - the resulting quota for the user on the namespaceIOExceptiondefault void preSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the current quota for the tableIOExceptiondefault void postSetTableQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and mastertableName - the name of the tablequotas - the resulting quota for the tableIOExceptiondefault void preSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the current quota for the namespaceIOExceptiondefault void postSetNamespaceQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masternamespace - the name of the namespacequotas - the resulting quota for the namespaceIOExceptiondefault void preSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masterregionServer - the name of the region serverquotas - the current quota for the region serverIOExceptiondefault void postSetRegionServerQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, String regionServer, GlobalQuotaSettings quotas) throws IOException
ctx - the environment to interact with the framework and masterregionServer - the name of the region serverquotas - the resulting quota for the region serverIOExceptiondefault void preMergeRegions(ObserverContext<MasterCoprocessorEnvironment> ctx, RegionInfo[] regionsToMerge) throws IOException
ctx - coprocessor environmentregionsToMerge - regions to be mergedIOExceptiondefault void postMergeRegions(ObserverContext<MasterCoprocessorEnvironment> c, RegionInfo[] regionsToMerge) throws IOException
c - coprocessor environmentregionsToMerge - regions to be mergedIOExceptiondefault void preMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - destination groupIOExceptiondefault void postMoveServersAndTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - name of groupIOExceptiondefault void preMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - destination groupIOExceptiondefault void postMoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers, String targetGroup) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to movetargetGroup - name of groupIOExceptiondefault void preMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and mastertables - set of tables to movetargetGroup - name of groupIOExceptiondefault void postMoveTables(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<TableName> tables, String targetGroup) throws IOException
ctx - the environment to interact with the framework and mastertables - set of tables to movetargetGroup - name of groupIOExceptiondefault void preAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void postAddRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void preRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void postRemoveRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String name) throws IOException
ctx - the environment to interact with the framework and mastername - group nameIOExceptiondefault void preBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOExceptiondefault void postBalanceRSGroup(ObserverContext<MasterCoprocessorEnvironment> ctx, String groupName, boolean balancerRan) throws IOException
ctx - the environment to interact with the framework and mastergroupName - group nameIOExceptiondefault void preRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException
ctx - the environment to interact with the framework and masterservers - set of decommissioned servers to removeIOExceptiondefault void postRemoveServers(ObserverContext<MasterCoprocessorEnvironment> ctx, Set<Address> servers) throws IOException
ctx - the environment to interact with the framework and masterservers - set of servers to removeIOExceptiondefault void preAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerpeerConfig - configuration for the replication peerIOExceptiondefault void postAddReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerpeerConfig - configuration for the replication peerIOExceptiondefault void preRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postRemoveReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postEnableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postDisableReplicationPeer(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postGetReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void preUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - peerId - a short name that identifies the peerIOExceptiondefault void postUpdateReplicationPeerConfig(ObserverContext<MasterCoprocessorEnvironment> ctx, String peerId, ReplicationPeerConfig peerConfig) throws IOException
ctx - the environment to interact with the framework and masterpeerId - a short name that identifies the peerIOExceptiondefault void preListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException
ctx - the environment to interact with the framework and masterregex - The regular expression to match peer idIOExceptiondefault void postListReplicationPeers(ObserverContext<MasterCoprocessorEnvironment> ctx, String regex) throws IOException
ctx - the environment to interact with the framework and masterregex - The regular expression to match peer idIOExceptiondefault void preRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postRequestLock(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace, TableName tableName, RegionInfo[] regionInfos, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tn, String description) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void postLockHeartbeat(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the environment to interact with the framework and masterIOExceptiondefault void preGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postGetClusterMetrics(ObserverContext<MasterCoprocessorEnvironment> ctx, ClusterMetrics status) throws IOException
IOExceptiondefault void preClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postClearDeadServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, List<ServerName> notClearedServers) throws IOException
IOExceptiondefault void preDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException
IOExceptiondefault void postDecommissionRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx, List<ServerName> servers, boolean offload) throws IOException
IOExceptiondefault void preListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void postListDecommissionedRegionServers(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
IOExceptiondefault void preRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException
IOExceptiondefault void postRecommissionRegionServer(ObserverContext<MasterCoprocessorEnvironment> ctx, ServerName server, List<byte[]> encodedRegionNames) throws IOException
IOExceptiondefault void preSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException
ctx - the coprocessor instance's environmentenable - the rpc throttle valueIOExceptiondefault void postSwitchRpcThrottle(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentoldValue - the previously rpc throttle valuenewValue - the newly rpc throttle valueIOExceptiondefault void preIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx) throws IOException
ctx - the coprocessor instance's environmentIOExceptiondefault void postIsRpcThrottleEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean rpcThrottleEnabled) throws IOException
ctx - the coprocessor instance's environmentrpcThrottleEnabled - the rpc throttle enabled valueIOExceptiondefault void preSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean enable) throws IOException
ctx - the coprocessor instance's environmentenable - the exceed throttle quota valueIOExceptiondefault void postSwitchExceedThrottleQuota(ObserverContext<MasterCoprocessorEnvironment> ctx, boolean oldValue, boolean newValue) throws IOException
ctx - the coprocessor instance's environmentoldValue - the previously exceed throttle quota valuenewValue - the newly exceed throttle quota valueIOExceptiondefault void preGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
ctx - the coprocessor instance's environmentuserPermission - the user and permissionsmergeExistingPermissions - True if merge with previous granted permissionsIOExceptiondefault void postGrant(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission, boolean mergeExistingPermissions) throws IOException
ctx - the coprocessor instance's environmentuserPermission - the user and permissionsmergeExistingPermissions - True if merge with previous granted permissionsIOExceptiondefault void preRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException
ctx - the coprocessor instance's environmentuserPermission - the user and permissionsIOExceptiondefault void postRevoke(ObserverContext<MasterCoprocessorEnvironment> ctx, UserPermission userPermission) throws IOException
ctx - the coprocessor instance's environmentuserPermission - the user and permissionsIOExceptiondefault void preGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException
ctx - the coprocessor instance's environmentuserName - the user name, null if get all user permissionsnamespace - the namespace, null if don't get namespace permissiontableName - the table name, null if don't get table permissionfamily - the table column family, null if don't get table family permissionqualifier - the table column qualifier, null if don't get table qualifier permissionIOException - if something went wrongdefault void postGetUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, String namespace, TableName tableName, byte[] family, byte[] qualifier) throws IOException
ctx - the coprocessor instance's environmentuserName - the user name, null if get all user permissionsnamespace - the namespace, null if don't get namespace permissiontableName - the table name, null if don't get table permissionfamily - the table column family, null if don't get table family permissionqualifier - the table column qualifier, null if don't get table qualifier permissionIOException - if something went wrongdefault void preHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException
IOExceptiondefault void postHasUserPermissions(ObserverContext<MasterCoprocessorEnvironment> ctx, String userName, List<Permission> permissions) throws IOException
ctx - the coprocessor instance's environmentuserName - the user namepermissions - the permission listIOExceptionCopyright © 2007–2020 The Apache Software Foundation. All rights reserved.