You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2017/11/21 06:19:30 UTC

hbase git commit: HBASE-19123 Purge 'complete' support from Coprocesor Observers

Repository: hbase
Updated Branches:
  refs/heads/branch-2 086a03797 -> 719a935b9


HBASE-19123 Purge 'complete' support from Coprocesor Observers


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/719a935b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/719a935b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/719a935b

Branch: refs/heads/branch-2
Commit: 719a935b9d13ab59708f74ddc39042d80b6d952f
Parents: 086a037
Author: Michael Stack <st...@apache.org>
Authored: Thu Nov 16 18:46:27 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Mon Nov 20 22:19:06 2017 -0800

----------------------------------------------------------------------
 .../hbase/coprocessor/CoprocessorHost.java      | 10 +--
 .../hbase/coprocessor/ObserverContext.java      | 12 ---
 .../hbase/coprocessor/ObserverContextImpl.java  | 38 +---------
 .../hbase/coprocessor/RegionObserver.java       | 79 --------------------
 .../hbase/mob/compactions/TestMobCompactor.java |  1 -
 5 files changed, 3 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/719a935b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
index 61c71cb..10e569b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/CoprocessorHost.java
@@ -559,8 +559,7 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
     }
 
     ObserverOperation(ObserverGetter<C, O> observerGetter, User user, boolean bypassable) {
-      super(user != null? user: RpcServer.getRequestUser().orElse(null),
-          bypassable, bypassable/*'completable': make completable same as bypassable*/);
+      super(user != null? user: RpcServer.getRequestUser().orElse(null), bypassable);
       this.observerGetter = observerGetter;
     }
 
@@ -678,10 +677,6 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
       }
       // Internal to shouldBypass, it checks if obeserverOperation#isBypassable().
       bypass |= observerOperation.shouldBypass();
-      // Internal to shouldComplete, it checks if obeserverOperation#isCompletable().
-      if (observerOperation.shouldComplete()) {
-        break;
-      }
       observerOperation.postEnvCall();
     }
     return bypass;
@@ -718,9 +713,6 @@ public abstract class CoprocessorHost<C extends Coprocessor, E extends Coprocess
         currentThread.setContextClassLoader(cl);
       }
       bypass |= observerOperation.shouldBypass();
-      if (observerOperation.shouldComplete()) {
-        break;
-      }
     }
 
     // Iterate the coprocessors and execute ObserverOperation's postEnvCall()

http://git-wip-us.apache.org/repos/asf/hbase/blob/719a935b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
index 5cbf4f6..ab611be 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContext.java
@@ -62,22 +62,10 @@ public interface ObserverContext<E extends CoprocessorEnvironment> {
    * that the replacement for the bypassed code takes care of all necessary
    * skipped concerns. Because those concerns can change at any point, such an
    * assumption is never safe.</p>
-   * @see #complete()
    */
   void bypass();
 
   /**
-   * Call to skip out on calling remaining coprocessors in current execution chain (there may be
-   * more than one coprocessor chained to a method call). Implies that this coprocessor's response
-   * is definitive.
-   * <p>Since hbase-2.0.0, only <code>complete</code> of 'bypassable' methods has an effect. See
-   * javadoc on the Coprocessor Observer method as to whether bypass (and thereby 'complete') is
-   * supported. This behavior of honoring only a subset of methods is new since hbase-2.0.0.
-   * @see #bypass()
-   */
-  void complete();
-
-  /**
    * Returns the active user for the coprocessor call. If an explicit {@code User} instance was
    * provided to the constructor, that will be returned, otherwise if we are in the context of an
    * RPC call, the remote user is used. May not be present if the execution is outside of an RPC

http://git-wip-us.apache.org/repos/asf/hbase/blob/719a935b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
index bdd6fec..b52c231 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/ObserverContextImpl.java
@@ -21,11 +21,9 @@ import java.util.Optional;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.hbase.CoprocessorEnvironment;
-import org.apache.hadoop.hbase.HBaseInterfaceAudience;
 import org.apache.hadoop.hbase.ipc.RpcServer;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.yetus.audience.InterfaceAudience;
-import org.apache.yetus.audience.InterfaceStability;
 
 /**
  * This is the only implementation of {@link ObserverContext}, which serves as the interface for
@@ -39,21 +37,15 @@ public class ObserverContextImpl<E extends CoprocessorEnvironment> implements Ob
    * Is this operation bypassable?
    */
   private final boolean bypassable;
-  /**
-   * Is this operation completable?
-   */
-  private boolean complete;
-  private final boolean completable;
   private final User caller;
 
   public ObserverContextImpl(User caller) {
-    this(caller, false, false);
+    this(caller, false);
   }
 
-  public ObserverContextImpl(User caller, boolean bypassable, boolean completable) {
+  public ObserverContextImpl(User caller, boolean bypassable) {
     this.caller = caller;
     this.bypassable = bypassable;
-    this.completable = completable;
   }
 
   public E getEnvironment() {
@@ -75,17 +67,6 @@ public class ObserverContextImpl<E extends CoprocessorEnvironment> implements Ob
     bypass = true;
   }
 
-  public boolean isCompleable() {
-    return this.completable;
-  };
-
-  public void complete() {
-    if (!this.completable) {
-      throw new UnsupportedOperationException("This method does not support 'complete'.");
-    }
-    complete = true;
-  }
-
   /**
    * @return {@code true}, if {@link ObserverContext#bypass()} was called by one of the loaded
    * coprocessors, {@code false} otherwise.
@@ -101,21 +82,6 @@ public class ObserverContextImpl<E extends CoprocessorEnvironment> implements Ob
     return false;
   }
 
-  /**
-   * @return {@code true}, if {@link ObserverContext#complete()} was called by one of the loaded
-   * coprocessors, {@code false} otherwise.
-   */
-  public boolean shouldComplete() {
-    if (!isCompleable()) {
-      return false;
-    }
-    if (complete) {
-      complete = false;
-      return true;
-    }
-    return false;
-  }
-
   public Optional<User> getCaller() {
     return Optional.ofNullable(caller);
   }

http://git-wip-us.apache.org/repos/asf/hbase/blob/719a935b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
index 4b8e3b8..7ac0a7e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/RegionObserver.java
@@ -310,9 +310,6 @@ public interface RegionObserver {
    * Called before the client performs a Get
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
    * @param c the environment provided by the region server
    * @param get the Get request
    * @param result The result to return to the client if default processing
@@ -325,9 +322,6 @@ public interface RegionObserver {
   /**
    * Called after the client performs a Get
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'result' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -341,9 +335,6 @@ public interface RegionObserver {
    * Called before the client tests for existence using a Get.
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
    * @param c the environment provided by the region server
    * @param get the Get request
    * @param exists the result returned by the region server
@@ -356,9 +347,6 @@ public interface RegionObserver {
 
   /**
    * Called after the client tests for existence using a Get.
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
    * @param c the environment provided by the region server
    * @param get the Get request
    * @param exists the result returned by the region server
@@ -374,9 +362,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -390,9 +375,6 @@ public interface RegionObserver {
   /**
    * Called after the client stores a value.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -408,9 +390,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -425,8 +404,6 @@ public interface RegionObserver {
    * Called before the server updates the timestamp for version delete with latest timestamp.
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
    * @param c the environment provided by the region server
    * @param mutation - the parent mutation associated with this delete cell
    * @param cell - The deleteColumn with latest version cell
@@ -443,9 +420,6 @@ public interface RegionObserver {
   /**
    * Called after the client deletes a value.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -523,9 +497,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -554,9 +525,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -579,9 +547,6 @@ public interface RegionObserver {
   /**
    * Called after checkAndPut
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'put' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -605,9 +570,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -635,9 +597,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -659,9 +618,6 @@ public interface RegionObserver {
   /**
    * Called after checkAndDelete
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'delete' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -685,9 +641,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -708,9 +661,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -725,9 +675,6 @@ public interface RegionObserver {
   /**
    * Called after Append
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'append' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -745,9 +692,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -768,8 +712,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    *
@@ -788,9 +730,6 @@ public interface RegionObserver {
   /**
    * Called after increment
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells in 'increment' beyond the life of this invocation.
    * If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -806,9 +745,6 @@ public interface RegionObserver {
   /**
    * Called before the client opens a new scanner.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
    * invocation. If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -821,9 +757,6 @@ public interface RegionObserver {
   /**
    * Called after the client opens a new scanner.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
    * invocation. If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -841,9 +774,6 @@ public interface RegionObserver {
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
    * invocation. If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -863,9 +793,6 @@ public interface RegionObserver {
   /**
    * Called after the client asks for the next row on a scanner.
    * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
-   * <p>
    * Note: Do not retain references to any Cells returned by scanner, beyond the life of this
    * invocation. If need a Cell reference for later use, copy the cell and use that.
    * @param c the environment provided by the region server
@@ -910,9 +837,6 @@ public interface RegionObserver {
    * Called before the client closes a scanner.
    * <p>
    * Call CoprocessorEnvironment#bypass to skip default actions
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
    * @param c the environment provided by the region server
    * @param s the scanner
    */
@@ -921,9 +845,6 @@ public interface RegionObserver {
 
   /**
    * Called after the client closes a scanner.
-   * <p>
-   * Call CoprocessorEnvironment#complete to skip any subsequent chained
-   * coprocessors
    * @param ctx the environment provided by the region server
    * @param s the scanner
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/719a935b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
index 3f4633a..54071d0 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/mob/compactions/TestMobCompactor.java
@@ -741,7 +741,6 @@ public class TestMobCompactor {
           candidates.remove(0);
         }
         c.bypass();
-        c.complete();
       }
     }
   }