Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-18169 Coprocessor fix and cleanup before 2.0.0 release
  3. HBASE-18298

RegionServerServices Interface cleanup for CP expose

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Closed
    • Critical
    • Resolution: Fixed
    • None
    • 2.0.0-alpha-4, 2.0.0
    • Coprocessors
    • None
    • Incompatible change, Reviewed
    • Hide
      We used to pass the RegionServerServices (RSS) which gave Coprocesosrs (CP) all sort of access to internal Server machinery. We now only allows the CP a subset of the RSS in the form of the CPRSS Interface. Particulars:

      Removed method getRegionServerServices from CP exposed RegionCoprocessorEnvironment and RegionServerCoprocessorEnvironment and replaced with getCoprocessorRegionServerServices. This returns a new interface CoprocessorRegionServerServices which is only a subset of RegionServerServices. With that below methods are no longer exposed for CPs
      WAL getWAL(HRegionInfo regionInfo)
      List<WAL> getWALs()
      FlushRequester getFlushRequester()
      RegionServerAccounting getRegionServerAccounting()
      RegionServerRpcQuotaManager getRegionServerRpcQuotaManager()
      SecureBulkLoadManager getSecureBulkLoadManager()
      RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager()
      void postOpenDeployTasks(final PostOpenDeployContext context)
      void postOpenDeployTasks(final Region r)
      boolean reportRegionStateTransition(final RegionStateTransitionContext context)
      boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris)
      boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris)
      RpcServerInterface getRpcServer()
      ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS()
      Leases getLeases()
      ExecutorService getExecutorService()
      Map<String, Region> getRecoveringRegions()
      public ServerNonceManager getNonceManager()
      boolean registerService(Service service)
      HeapMemoryManager getHeapMemoryManager()
      double getCompactionPressure()
      ThroughputController getFlushThroughputController()
      double getFlushPressure()
      MetricsRegionServer getMetrics()
      EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort)
      void unassign(byte[] regionName)
      Configuration getConfiguration()
      ZooKeeperWatcher getZooKeeper()
      ClusterConnection getClusterConnection()
      MetaTableLocator getMetaTableLocator()
      CoordinatedStateManager getCoordinatedStateManager()
      ChoreService getChoreService()
      void stop(String why)
      void abort(String why, Throwable e)
      boolean isAborted()
      void updateRegionFavoredNodesMapping(String encodedRegionName, List<ServerName> favoredNodes)
      InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName)
      void addToOnlineRegions(Region region)
      boolean removeFromOnlineRegions(final Region r, ServerName destination)

      Also 3 methods name have been changed
      List<Region> getOnlineRegions(TableName tableName) -> List<Region> getRegions(TableName tableName)
      List<Region> getOnlineRegions() -> List<Region> getRegions()
      Region getFromOnlineRegions(final String encodedRegionName) -> Region getRegion(final String encodedRegionName)
      Show
      We used to pass the RegionServerServices (RSS) which gave Coprocesosrs (CP) all sort of access to internal Server machinery. We now only allows the CP a subset of the RSS in the form of the CPRSS Interface. Particulars: Removed method getRegionServerServices from CP exposed RegionCoprocessorEnvironment and RegionServerCoprocessorEnvironment and replaced with getCoprocessorRegionServerServices. This returns a new interface CoprocessorRegionServerServices which is only a subset of RegionServerServices. With that below methods are no longer exposed for CPs WAL getWAL(HRegionInfo regionInfo) List<WAL> getWALs() FlushRequester getFlushRequester() RegionServerAccounting getRegionServerAccounting() RegionServerRpcQuotaManager getRegionServerRpcQuotaManager() SecureBulkLoadManager getSecureBulkLoadManager() RegionServerSpaceQuotaManager getRegionServerSpaceQuotaManager() void postOpenDeployTasks(final PostOpenDeployContext context) void postOpenDeployTasks(final Region r) boolean reportRegionStateTransition(final RegionStateTransitionContext context) boolean reportRegionStateTransition(TransitionCode code, long openSeqNum, HRegionInfo... hris) boolean reportRegionStateTransition(TransitionCode code, HRegionInfo... hris) RpcServerInterface getRpcServer() ConcurrentMap<byte[], Boolean> getRegionsInTransitionInRS() Leases getLeases() ExecutorService getExecutorService() Map<String, Region> getRecoveringRegions() public ServerNonceManager getNonceManager() boolean registerService(Service service) HeapMemoryManager getHeapMemoryManager() double getCompactionPressure() ThroughputController getFlushThroughputController() double getFlushPressure() MetricsRegionServer getMetrics() EntityLock regionLock(List<HRegionInfo> regionInfos, String description, Abortable abort) void unassign(byte[] regionName) Configuration getConfiguration() ZooKeeperWatcher getZooKeeper() ClusterConnection getClusterConnection() MetaTableLocator getMetaTableLocator() CoordinatedStateManager getCoordinatedStateManager() ChoreService getChoreService() void stop(String why) void abort(String why, Throwable e) boolean isAborted() void updateRegionFavoredNodesMapping(String encodedRegionName, List<ServerName> favoredNodes) InetSocketAddress[] getFavoredNodesForRegion(String encodedRegionName) void addToOnlineRegions(Region region) boolean removeFromOnlineRegions(final Region r, ServerName destination) Also 3 methods name have been changed List<Region> getOnlineRegions(TableName tableName) -> List<Region> getRegions(TableName tableName) List<Region> getOnlineRegions() -> List<Region> getRegions() Region getFromOnlineRegions(final String encodedRegionName) -> Region getRegion(final String encodedRegionName)

    Attachments

      1. HBASE-18298_V2.patch
        109 kB
        Anoop Sam John
      2. HBASE-18298_V3.patch
        112 kB
        Anoop Sam John
      3. HBASE-18298_V4.patch
        27 kB
        Anoop Sam John
      4. HBASE-18298_V5.patch
        39 kB
        Anoop Sam John
      5. HBASE-18298_V6.patch
        39 kB
        Anoop Sam John
      6. HBASE-18298_V7.patch
        140 kB
        Anoop Sam John
      7. HBASE-18298_V7.patch
        140 kB
        Anoop Sam John
      8. HBASE-18298.patch
        108 kB
        Anoop Sam John

      Issue Links

        Activity

          People

            anoop.hbase Anoop Sam John
            anoop.hbase Anoop Sam John
            Votes:
            0 Vote for this issue
            Watchers:
            11 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: