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/19 03:09:02 UTC

[1/2] hbase git commit: HBASE-19299 Assert only one Connection is constructed when calculating splits in a MultiTableInputFormat

Repository: hbase
Updated Branches:
  refs/heads/master 777b653b4 -> b4fbf5fe1


HBASE-19299 Assert only one Connection is constructed when calculating splits in a MultiTableInputFormat

Adds a test suite that has one test in it. Does a bunch of mocking so
getSplits can run. Has counter in mocked Connection constructor so can
count how many Connections made during call to getSplits. Verified that
assert fails if more than one Connection made.


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

Branch: refs/heads/master
Commit: b4fbf5fe18bc9247106f674580666096fd34d3fa
Parents: 08544e5
Author: Michael Stack <st...@apache.org>
Authored: Sat Nov 18 16:03:16 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Nov 18 16:04:27 2017 -0800

----------------------------------------------------------------------
 .../TestMultiTableInputFormatBase.java          | 244 +++++++++++++++++++
 1 file changed, 244 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/b4fbf5fe/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
new file mode 100644
index 0000000..47d7c0c
--- /dev/null
+++ b/hbase-mapreduce/src/test/java/org/apache/hadoop/hbase/mapreduce/TestMultiTableInputFormatBase.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.mapreduce;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CategoryBasedTimeout;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.BufferedMutator;
+import org.apache.hadoop.hbase.client.BufferedMutatorParams;
+import org.apache.hadoop.hbase.client.ClusterConnection;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionLocator;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableBuilder;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.junit.rules.TestRule;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.hadoop.hbase.client.Scan.SCAN_ATTRIBUTES_TABLE_NAME;
+
+/**
+ * Tests of MultiTableInputFormatBase.
+ */
+@Category({SmallTests.class})
+public class TestMultiTableInputFormatBase {
+  @Rule public final TestName name = new TestName();
+  @Rule public final TestRule timeout = CategoryBasedTimeout.builder()
+      .withTimeout(this.getClass())
+      .withLookingForStuckThread(true)
+      .build();
+
+  /**
+   * Test getSplits only puts up one Connection.
+   * In past it has put up many Connections. Each Connection setup comes with a fresh new cache
+   * so we have to do fresh hit on hbase:meta. Should only do one Connection when doing getSplits
+   * even if a MultiTableInputFormat.
+   * @throws IOException
+   */
+  @Test
+  public void testMRSplitsConnectionCount() throws IOException {
+    // Make instance of MTIFB.
+    MultiTableInputFormatBase mtif = new MultiTableInputFormatBase() {
+      @Override
+      public RecordReader<ImmutableBytesWritable, Result> createRecordReader(InputSplit split,
+          TaskAttemptContext context)
+      throws IOException, InterruptedException {
+        return super.createRecordReader(split, context);
+      }
+    };
+    // Pass it a mocked JobContext. Make the JC return our Configuration.
+    // Load the Configuration so it returns our special Connection so we can interpolate
+    // canned responses.
+    JobContext mockedJobContext = Mockito.mock(JobContext.class);
+    Configuration c = HBaseConfiguration.create();
+    c.set(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL, MRSplitsConnection.class.getName());
+    Mockito.when(mockedJobContext.getConfiguration()).thenReturn(c);
+    // Invent a bunch of scans. Have each Scan go against a different table so a good spread.
+    List<Scan> scans = new ArrayList<>();
+    for (int i = 0; i < 10; i++) {
+      Scan scan = new Scan();
+      String tableName = this.name.getMethodName() + i;
+      scan.setAttribute(SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(tableName));
+      scans.add(scan);
+    }
+    mtif.setScans(scans);
+    // Get splits. Assert that that more than one.
+    List<InputSplit> splits = mtif.getSplits(mockedJobContext);
+    Assert.assertTrue(splits.size() > 0);
+    // Assert only one Connection was made (see the static counter we have in the mocked
+    // Connection MRSplitsConnection Constructor.
+    Assert.assertEquals(1, MRSplitsConnection.creations.get());
+  }
+
+  /**
+   * Connection to use above in Test.
+   */
+  public static class MRSplitsConnection implements Connection {
+    private final Configuration configuration;
+    static final AtomicInteger creations = new AtomicInteger(0);
+
+    MRSplitsConnection (Configuration conf, ExecutorService pool, User user) throws IOException {
+      this.configuration = conf;
+      creations.incrementAndGet();
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {
+
+    }
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public Configuration getConfiguration() {
+      return this.configuration;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(TableName tableName) throws IOException {
+      return null;
+    }
+
+    @Override
+    public BufferedMutator getBufferedMutator(BufferedMutatorParams params) throws IOException {
+      return null;
+    }
+
+    @Override
+    public RegionLocator getRegionLocator(final TableName tableName) throws IOException {
+      // Make up array of start keys. We start off w/ empty byte array.
+      final byte [][] startKeys = new byte [][] {HConstants.EMPTY_BYTE_ARRAY,
+          Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"),
+          Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
+          Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
+          Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
+          Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"),
+          Bytes.toBytes("qqq"), Bytes.toBytes("rrr"), Bytes.toBytes("sss"),
+          Bytes.toBytes("ttt"), Bytes.toBytes("uuu"), Bytes.toBytes("vvv"),
+          Bytes.toBytes("zzz")};
+      // Make an array of end keys. We end with the empty byte array.
+      final byte [][] endKeys = new byte[][] {
+          Bytes.toBytes("aaaa"), Bytes.toBytes("bbb"),
+          Bytes.toBytes("ccc"), Bytes.toBytes("ddd"), Bytes.toBytes("eee"),
+          Bytes.toBytes("fff"), Bytes.toBytes("ggg"), Bytes.toBytes("hhh"),
+          Bytes.toBytes("iii"), Bytes.toBytes("lll"), Bytes.toBytes("mmm"),
+          Bytes.toBytes("nnn"), Bytes.toBytes("ooo"), Bytes.toBytes("ppp"),
+          Bytes.toBytes("qqq"), Bytes.toBytes("rrr"), Bytes.toBytes("sss"),
+          Bytes.toBytes("ttt"), Bytes.toBytes("uuu"), Bytes.toBytes("vvv"),
+          Bytes.toBytes("zzz"),
+          HConstants.EMPTY_BYTE_ARRAY};
+      // Now make a map of start keys to HRegionLocations. Let the server namber derive from
+      // the start key.
+      final Map<byte [], HRegionLocation> map =
+          new TreeMap<byte [], HRegionLocation>(Bytes.BYTES_COMPARATOR);
+      for (byte [] startKey: startKeys) {
+        HRegionLocation hrl = new HRegionLocation(
+            RegionInfoBuilder.newBuilder(tableName).setStartKey(startKey).build(),
+            ServerName.valueOf(Bytes.toString(startKey), 0, 0));
+        map.put(startKey, hrl);
+      }
+      // Get a list of the locations.
+      final List<HRegionLocation> locations = new ArrayList<HRegionLocation>(map.values());
+      // Now make a RegionLocator mock backed by the abpve map and list of locations.
+      RegionLocator mockedRegionLocator = Mockito.mock(RegionLocator.class);
+      Mockito.when(mockedRegionLocator.getRegionLocation(Mockito.any(byte [].class),
+            Mockito.anyBoolean())).
+          thenAnswer(new Answer<HRegionLocation>() {
+            @Override
+            public HRegionLocation answer(InvocationOnMock invocationOnMock) throws Throwable {
+              Object [] args = invocationOnMock.getArguments();
+              byte [] key = (byte [])args[0];
+              return map.get(key);
+            }
+          });
+      Mockito.when(mockedRegionLocator.getAllRegionLocations()).thenReturn(locations);
+      Mockito.when(mockedRegionLocator.getStartEndKeys()).
+          thenReturn(new Pair<byte [][], byte[][]>(startKeys, endKeys));
+      Mockito.when(mockedRegionLocator.getName()).thenReturn(tableName);
+      return mockedRegionLocator;
+    }
+
+    @Override
+    public Admin getAdmin() throws IOException {
+      Admin admin = Mockito.mock(Admin.class);
+      Mockito.when(admin.getConfiguration()).thenReturn(getConfiguration());
+      return admin;
+    }
+
+    @Override
+    public Table getTable(TableName tableName) throws IOException {
+      Table table = Mockito.mock(Table.class);
+      Mockito.when(table.getName()).thenReturn(tableName);
+      return table;
+    }
+
+    @Override
+    public void close() throws IOException {
+
+    }
+
+    @Override
+    public boolean isClosed() {
+      return false;
+    }
+
+    @Override
+    public TableBuilder getTableBuilder(TableName tableName, ExecutorService pool) {
+      return Mockito.mock(TableBuilder.class);
+    }
+  }
+}


[2/2] hbase git commit: HBASE-19123 Purge 'complete' support from Coprocesor Observers

Posted by st...@apache.org.
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/08544e54
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/08544e54
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/08544e54

Branch: refs/heads/master
Commit: 08544e54a999df16cb0cef7cf45a17da1eeef42d
Parents: 777b653
Author: Michael Stack <st...@apache.org>
Authored: Thu Nov 16 18:46:27 2017 -0800
Committer: Michael Stack <st...@apache.org>
Committed: Sat Nov 18 16:04:27 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/08544e54/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/08544e54/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/08544e54/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/08544e54/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/08544e54/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();
       }
     }
   }