You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2016/06/08 03:42:24 UTC

[1/4] hbase git commit: HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)

Repository: hbase
Updated Branches:
  refs/heads/branch-1 7fed7a8f4 -> 602b2839c
  refs/heads/branch-1.3 600d10a8b -> 2bba03cf5
  refs/heads/master 3a95552cf -> d05a3722c


HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)


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

Branch: refs/heads/master
Commit: d05a3722c8347363eb04a3e5457d13ae5d0d6de6
Parents: 3a95552
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Jun 7 20:12:51 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Jun 7 20:12:51 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |  76 ++++-
 .../MiniBatchOperationInProgress.java           |  29 +-
 .../regionserver/MultiRowMutationProcessor.java |   2 +-
 ...erverForAddingMutationsFromCoprocessors.java | 282 +++++++++++++++++++
 4 files changed, 370 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d05a3722/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 6522fde..9c966cd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -3134,6 +3134,35 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         if (coprocessorHost.preBatchMutate(miniBatchOp)) {
           return 0L;
+        } else {
+          for (int i = firstIndex; i < lastIndexExclusive; i++) {
+            if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
+              // lastIndexExclusive was incremented above.
+              continue;
+            }
+            // we pass (i - firstIndex) below since the call expects a relative index
+            Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - firstIndex);
+            if (cpMutations == null) {
+              continue;
+            }
+            // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
+            for (int j = 0; j < cpMutations.length; j++) {
+              Mutation cpMutation = cpMutations[j];
+              Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
+              checkAndPrepareMutation(cpMutation, replay, cpFamilyMap, now);
+
+              // Acquire row locks. If not, the whole batch will fail.
+              acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true));
+
+              if (cpMutation.getDurability() == Durability.SKIP_WAL) {
+                recordMutationWithoutWal(cpFamilyMap);
+              }
+
+              // Returned mutations from coprocessor correspond to the Mutation at index i. We can
+              // directly add the cells from those mutations to the familyMaps of this mutation.
+              mergeFamilyMaps(familyMaps[i], cpFamilyMap); // will get added to the memstore later
+            }
+          }
         }
       }
 
@@ -3310,9 +3339,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // call the coprocessor hook to do any finalization steps
         // after the put is done
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
-            new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
-                batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex,
-                lastIndexExclusive);
+          new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
+          batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
       }
 
@@ -3320,6 +3348,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
+      Map<byte[], List<Cell>> toBeMerged) {
+    for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
+      List<Cell> cells = familyMap.get(entry.getKey());
+      if (cells == null) {
+        familyMap.put(entry.getKey(), entry.getValue());
+      } else {
+        cells.addAll(entry.getValue());
+      }
+    }
+  }
+
   private void appendCurrentNonces(final Mutation mutation, final boolean replay,
       final WALEdit walEdit, final long now, final long currentNonceGroup, final long currentNonce)
   throws IOException {
@@ -3348,18 +3388,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     familyMaps[lastIndexExclusive] = familyMap;
 
     try {
-      if (mutation instanceof Put) {
-        // Check the families in the put. If bad, skip this one.
-        if (batchOp.isInReplay()) {
-          removeNonExistentColumnFamilyForReplay(familyMap);
-        } else {
-          checkFamilies(familyMap.keySet());
-        }
-        checkTimestamps(mutation.getFamilyCellMap(), now);
-      } else {
-        prepareDelete((Delete)mutation);
-      }
-      checkRow(mutation.getRow(), "doMiniBatchMutation");
+      checkAndPrepareMutation(mutation, batchOp.isInReplay(), familyMap, now);
     } catch (NoSuchColumnFamilyException nscf) {
       LOG.warn("No such column family in batch mutation", nscf);
       batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
@@ -3379,6 +3408,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     return skip;
   }
 
+  private void checkAndPrepareMutation(Mutation mutation, boolean replay,
+      final Map<byte[], List<Cell>> familyMap, final long now)
+          throws IOException {
+    if (mutation instanceof Put) {
+      // Check the families in the put. If bad, skip this one.
+      if (replay) {
+        removeNonExistentColumnFamilyForReplay(familyMap);
+      } else {
+        checkFamilies(familyMap.keySet());
+      }
+      checkTimestamps(mutation.getFamilyCellMap(), now);
+    } else {
+      prepareDelete((Delete)mutation);
+    }
+    checkRow(mutation.getRow(), "doMiniBatchMutation");
+  }
+
   /**
    * During replay, there could exist column families which are removed between region server
    * failure and replay

http://git-wip-us.apache.org/repos/asf/hbase/blob/d05a3722/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
index 2b12dec..cdbecac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
@@ -18,20 +18,22 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 
 /**
  * Wraps together the mutations which are applied as a batch to the region and their operation
- * status and WALEdits. 
+ * status and WALEdits.
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#preBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#postBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @param T Pair&lt;Mutation, Integer&gt; pair of Mutations and associated rowlock ids .
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate("Coprocessors")
 public class MiniBatchOperationInProgress<T> {
   private final T[] operations;
+  private Mutation[][] operationsFromCoprocessors;
   private final OperationStatus[] retCodeDetails;
   private final WALEdit[] walEditsFromCoprocessors;
   private final int firstIndex;
@@ -63,7 +65,7 @@ public class MiniBatchOperationInProgress<T> {
 
   /**
    * Sets the status code for the operation(Mutation) at the specified position.
-   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} 
+   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
    * can make HRegion to skip Mutations.
    * @param index
    * @param opStatus
@@ -103,4 +105,25 @@ public class MiniBatchOperationInProgress<T> {
     }
     return this.firstIndex + index;
   }
+
+  /**
+   * Add more Mutations corresponding to the Mutation at the given index to be committed atomically
+   * in the same batch. These mutations are applied to the WAL and applied to the memstore as well.
+   * The timestamp of the cells in the given Mutations MUST be obtained from the original mutation.
+   *
+   * @param index the index that corresponds to the original mutation index in the batch
+   * @param newOperations the Mutations to add
+   */
+  public void addOperationsFromCP(int index, Mutation[] newOperations) {
+    if (this.operationsFromCoprocessors == null) {
+      // lazy allocation to save on object allocation in case this is not used
+      this.operationsFromCoprocessors = new Mutation[operations.length][];
+    }
+    this.operationsFromCoprocessors[getAbsoluteIndex(index)] = newOperations;
+  }
+
+  public Mutation[] getOperationsFromCoprocessors(int index) {
+    return operationsFromCoprocessors == null ? null :
+        operationsFromCoprocessors[getAbsoluteIndex(index)];
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/d05a3722/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
index 1947a1b..995ea93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
@@ -60,7 +60,7 @@ MultiRowMutationProcessorResponse> {
   public boolean readOnly() {
     return false;
   }
-  
+
   @Override
   public MultiRowMutationProcessorResponse getResult() {
     return MultiRowMutationProcessorResponse.getDefaultInstance();

http://git-wip-us.apache.org/repos/asf/hbase/blob/d05a3722/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
new file mode 100644
index 0000000..98e930a
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -0,0 +1,282 @@
+/**
+ * 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.coprocessor;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+
+@Category(MediumTests.class)
+public class TestRegionObserverForAddingMutationsFromCoprocessors {
+
+  private static final Log LOG
+    = LogFactory.getLog(TestRegionObserverForAddingMutationsFromCoprocessors.class);
+
+  private static HBaseTestingUtility util;
+  private static final byte[] dummy = Bytes.toBytes("dummy");
+  private static final byte[] row1 = Bytes.toBytes("r1");
+  private static final byte[] row2 = Bytes.toBytes("r2");
+  private static final byte[] row3 = Bytes.toBytes("r3");
+  private static final byte[] test = Bytes.toBytes("test");
+
+  @Rule
+  public TestName name = new TestName();
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, TestWALObserver.class.getName());
+    util = new HBaseTestingUtility(conf);
+    util.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  private void createTable(String coprocessor) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(tableName)
+        .addFamily(new HColumnDescriptor(dummy))
+        .addFamily(new HColumnDescriptor(test))
+        .addCoprocessor(coprocessor);
+    util.getAdmin().createTable(htd);
+  }
+
+  /**
+   * Test various multiput operations.
+   * @throws Exception
+   */
+  @Test
+  public void testMulti() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+  }
+
+  /**
+   * Tests that added mutations from coprocessors end up in the WAL.
+   */
+  @Test
+  public void testCPMutationsAreWrittenToWALEdit() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+
+    assertNotNull(TestWALObserver.savedEdit);
+    assertEquals(4, TestWALObserver.savedEdit.getCells().size());
+  }
+
+  private static void assertRowCount(Table t, int expected) throws IOException {
+    try (ResultScanner scanner = t.getScanner(new Scan())) {
+      int i = 0;
+      for (Result r: scanner) {
+        LOG.info(r.toString());
+        i++;
+      }
+      assertEquals(expected, i);
+    }
+  }
+
+  @Test
+  public void testDeleteCell() throws Exception {
+    createTable(TestDeleteCellCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteFamily() throws Exception {
+    createTable(TestDeleteFamilyCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addFamily(test)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteRow() throws Exception {
+    createTable(TestDeleteRowCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  public static class TestMultiMutationCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+      List<Cell> cells = mut.getFamilyCellMap().get(test);
+      Put[] puts = new Put[] {
+          new Put(row1).addColumn(test, dummy, cells.get(0).getTimestamp(),
+            Bytes.toBytes("cpdummy")),
+          new Put(row2).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+          new Put(row3).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+      };
+      LOG.info("Putting:" + puts);
+      miniBatchOp.addOperationsFromCP(0, puts);
+    }
+  }
+
+  public static class TestDeleteCellCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addColumns(test, dummy, cells.get(0).getTimestamp()),
+            new Delete(row2).addColumns(test, dummy, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteFamilyCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addFamily(test, cells.get(0).getTimestamp()),
+            new Delete(row2).addFamily(test, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteRowCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1, cells.get(0).getTimestamp()),
+            new Delete(row2, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestWALObserver extends BaseWALObserver {
+    static WALEdit savedEdit = null;
+    @Override
+    public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
+        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
+        savedEdit = logEdit;
+      }
+      super.postWALWrite(ctx, info, logKey, logEdit);
+    }
+  }
+}


[3/4] hbase git commit: HBASE-15714 We are calling checkRow() twice in doMiniBatchMutation()

Posted by en...@apache.org.
HBASE-15714 We are calling checkRow() twice in doMiniBatchMutation()


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

Branch: refs/heads/branch-1.3
Commit: fd31507048b1d0151297989db155d680cac5ed71
Parents: 600d10a
Author: chenheng <ch...@apache.org>
Authored: Tue May 3 12:45:18 2016 +1000
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Jun 7 20:39:14 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      | 22 ++++++++++++--------
 .../hbase/regionserver/TestAtomicOperation.java |  4 ++--
 2 files changed, 15 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fd315070/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index cd0cbca..3de8838 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -3114,7 +3114,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // get the next one.
         RowLock rowLock = null;
         try {
-          rowLock = getRowLock(mutation.getRow(), true);
+          rowLock = getRowLockInternal(mutation.getRow(), true);
         } catch (IOException ioe) {
           LOG.warn("Failed getting lock in batch put, row="
             + Bytes.toStringBinary(mutation.getRow()), ioe);
@@ -3467,9 +3467,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       Get get = new Get(row);
       checkFamily(family);
       get.addColumn(family, qualifier);
-
+      checkRow(row, "checkAndMutate");
       // Lock row - note that doBatchMutate will relock this row if called
-      RowLock rowLock = getRowLock(get.getRow());
+      RowLock rowLock = getRowLockInternal(get.getRow(), false);
       // wait for all previous transactions to complete (with lock held)
       mvcc.await();
       try {
@@ -3577,9 +3577,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       Get get = new Get(row);
       checkFamily(family);
       get.addColumn(family, qualifier);
-
+      checkRow(row, "checkAndRowMutate");
       // Lock row - note that doBatchMutate will relock this row if called
-      RowLock rowLock = getRowLock(get.getRow());
+      RowLock rowLock = getRowLockInternal(get.getRow(), false);
       // wait for all previous transactions to complete (with lock held)
       mvcc.await();
       try {
@@ -5214,6 +5214,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
   public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
     // Make sure the row is inside of this region before getting the lock for it.
     checkRow(row, "row lock");
+    return getRowLockInternal(row, readLock);
+  }
+
+  protected RowLock getRowLockInternal(byte[] row, boolean readLock) throws IOException {
     // create an object to use a a key in the row lock map
     HashedBytes rowKey = new HashedBytes(row);
 
@@ -7046,7 +7050,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       for (byte[] row : rowsToLock) {
         // Attempt to lock all involved rows, throw if any lock times out
         // use a writer lock for mixed reads and writes
-        acquiredRowLocks.add(getRowLock(row));
+        acquiredRowLocks.add(getRowLockInternal(row, false));
       }
       // 3. Region lock
       lock(this.updatesLock.readLock(), acquiredRowLocks.size() == 0 ? 1 : acquiredRowLocks.size());
@@ -7283,7 +7287,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     WALKey walKey = null;
     boolean doRollBackMemstore = false;
     try {
-      rowLock = getRowLock(row);
+      rowLock = getRowLockInternal(row, false);
       assert rowLock != null;
       try {
         lock(this.updatesLock.readLock());
@@ -7573,7 +7577,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     // changing it. These latter increments by zero are NOT added to the WAL.
     List<Cell> allKVs = new ArrayList<Cell>(increment.size());
     Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
-    RowLock rowLock = getRowLock(increment.getRow());
+    RowLock rowLock = getRowLockInternal(increment.getRow(), false);
     try {
       lock(this.updatesLock.readLock());
       try {
@@ -7669,7 +7673,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     List<Cell> memstoreCells = new ArrayList<Cell>();
     Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
     try {
-      rowLock = getRowLock(increment.getRow());
+      rowLock = getRowLockInternal(increment.getRow(), false);
       long txid = 0;
       try {
         lock(this.updatesLock.readLock());

http://git-wip-us.apache.org/repos/asf/hbase/blob/fd315070/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
index 4763c55..e49c265 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestAtomicOperation.java
@@ -685,11 +685,11 @@ public class TestAtomicOperation {
     }
 
     @Override
-    public RowLock getRowLock(final byte[] row, boolean readLock) throws IOException {
+    public RowLock getRowLockInternal(final byte[] row, boolean readLock) throws IOException {
       if (testStep == TestStep.CHECKANDPUT_STARTED) {
         latch.countDown();
       }
-      return new WrappedRowLock(super.getRowLock(row, readLock));
+      return new WrappedRowLock(super.getRowLockInternal(row, readLock));
     }
 
     public class WrappedRowLock implements RowLock {


[4/4] hbase git commit: HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)

Posted by en...@apache.org.
HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java


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

Branch: refs/heads/branch-1.3
Commit: 2bba03cf5ad925e8644cea0d22180eb527a9b067
Parents: fd31507
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Jun 7 20:12:51 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Jun 7 20:39:30 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |  80 ++++--
 .../MiniBatchOperationInProgress.java           |  29 +-
 .../regionserver/MultiRowMutationProcessor.java |   2 +-
 ...erverForAddingMutationsFromCoprocessors.java | 282 +++++++++++++++++++
 4 files changed, 373 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2bba03cf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 3de8838..04f7f81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -3078,18 +3078,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
 
         try {
-          if (isPutMutation) {
-            // Check the families in the put. If bad, skip this one.
-            if (isInReplay) {
-              removeNonExistentColumnFamilyForReplay(familyMap);
-            } else {
-              checkFamilies(familyMap.keySet());
-            }
-            checkTimestamps(mutation.getFamilyCellMap(), now);
-          } else {
-            prepareDelete((Delete) mutation);
-          }
-          checkRow(mutation.getRow(), "doMiniBatchMutation");
+          checkAndPrepareMutation(mutation, batchOp.isInReplay(), familyMap, now);
         } catch (NoSuchColumnFamilyException nscf) {
           LOG.warn("No such column family in batch mutation", nscf);
           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
@@ -3197,7 +3186,38 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
-        if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
+        if (coprocessorHost.preBatchMutate(miniBatchOp)) {
+          return 0L;
+        } else {
+          for (int i = firstIndex; i < lastIndexExclusive; i++) {
+            if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
+              // lastIndexExclusive was incremented above.
+              continue;
+            }
+            // we pass (i - firstIndex) below since the call expects a relative index
+            Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - firstIndex);
+            if (cpMutations == null) {
+              continue;
+            }
+            // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
+            for (int j = 0; j < cpMutations.length; j++) {
+              Mutation cpMutation = cpMutations[j];
+              Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
+              checkAndPrepareMutation(cpMutation, isInReplay, cpFamilyMap, now);
+
+              // Acquire row locks. If not, the whole batch will fail.
+              acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true));
+
+              if (cpMutation.getDurability() == Durability.SKIP_WAL) {
+                recordMutationWithoutWal(cpFamilyMap);
+              }
+
+              // Returned mutations from coprocessor correspond to the Mutation at index i. We can
+              // directly add the cells from those mutations to the familyMaps of this mutation.
+              mergeFamilyMaps(familyMaps[i], cpFamilyMap); // will get added to the memstore later
+            }
+          }
+        }
       }
 
       // ------------------------------------
@@ -3421,9 +3441,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // call the coprocessor hook to do any finalization steps
         // after the put is done
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
-            new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
-                batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex,
-                lastIndexExclusive);
+          new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
+          batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
       }
 
@@ -3431,6 +3450,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
+      Map<byte[], List<Cell>> toBeMerged) {
+    for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
+      List<Cell> cells = familyMap.get(entry.getKey());
+      if (cells == null) {
+        familyMap.put(entry.getKey(), entry.getValue());
+      } else {
+        cells.addAll(entry.getValue());
+      }
+    }
+  }
+
   /**
    * Returns effective durability from the passed durability and
    * the table descriptor.
@@ -3860,6 +3891,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void checkAndPrepareMutation(Mutation mutation, boolean replay,
+      final Map<byte[], List<Cell>> familyMap, final long now)
+          throws IOException {
+    if (mutation instanceof Put) {
+      // Check the families in the put. If bad, skip this one.
+      if (replay) {
+        removeNonExistentColumnFamilyForReplay(familyMap);
+      } else {
+        checkFamilies(familyMap.keySet());
+      }
+      checkTimestamps(mutation.getFamilyCellMap(), now);
+    } else {
+      prepareDelete((Delete)mutation);
+    }
+    checkRow(mutation.getRow(), "doMiniBatchMutation");
+  }
+
   /**
    * During replay, there could exist column families which are removed between region server
    * failure and replay

http://git-wip-us.apache.org/repos/asf/hbase/blob/2bba03cf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
index 2b12dec..cdbecac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
@@ -18,20 +18,22 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 
 /**
  * Wraps together the mutations which are applied as a batch to the region and their operation
- * status and WALEdits. 
+ * status and WALEdits.
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#preBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#postBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @param T Pair&lt;Mutation, Integer&gt; pair of Mutations and associated rowlock ids .
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate("Coprocessors")
 public class MiniBatchOperationInProgress<T> {
   private final T[] operations;
+  private Mutation[][] operationsFromCoprocessors;
   private final OperationStatus[] retCodeDetails;
   private final WALEdit[] walEditsFromCoprocessors;
   private final int firstIndex;
@@ -63,7 +65,7 @@ public class MiniBatchOperationInProgress<T> {
 
   /**
    * Sets the status code for the operation(Mutation) at the specified position.
-   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} 
+   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
    * can make HRegion to skip Mutations.
    * @param index
    * @param opStatus
@@ -103,4 +105,25 @@ public class MiniBatchOperationInProgress<T> {
     }
     return this.firstIndex + index;
   }
+
+  /**
+   * Add more Mutations corresponding to the Mutation at the given index to be committed atomically
+   * in the same batch. These mutations are applied to the WAL and applied to the memstore as well.
+   * The timestamp of the cells in the given Mutations MUST be obtained from the original mutation.
+   *
+   * @param index the index that corresponds to the original mutation index in the batch
+   * @param newOperations the Mutations to add
+   */
+  public void addOperationsFromCP(int index, Mutation[] newOperations) {
+    if (this.operationsFromCoprocessors == null) {
+      // lazy allocation to save on object allocation in case this is not used
+      this.operationsFromCoprocessors = new Mutation[operations.length][];
+    }
+    this.operationsFromCoprocessors[getAbsoluteIndex(index)] = newOperations;
+  }
+
+  public Mutation[] getOperationsFromCoprocessors(int index) {
+    return operationsFromCoprocessors == null ? null :
+        operationsFromCoprocessors[getAbsoluteIndex(index)];
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2bba03cf/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
index 1947a1b..995ea93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
@@ -60,7 +60,7 @@ MultiRowMutationProcessorResponse> {
   public boolean readOnly() {
     return false;
   }
-  
+
   @Override
   public MultiRowMutationProcessorResponse getResult() {
     return MultiRowMutationProcessorResponse.getDefaultInstance();

http://git-wip-us.apache.org/repos/asf/hbase/blob/2bba03cf/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
new file mode 100644
index 0000000..0663b0b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -0,0 +1,282 @@
+/**
+ * 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.coprocessor;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+
+@Category(MediumTests.class)
+public class TestRegionObserverForAddingMutationsFromCoprocessors {
+
+  private static final Log LOG
+    = LogFactory.getLog(TestRegionObserverForAddingMutationsFromCoprocessors.class);
+
+  private static HBaseTestingUtility util;
+  private static final byte[] dummy = Bytes.toBytes("dummy");
+  private static final byte[] row1 = Bytes.toBytes("r1");
+  private static final byte[] row2 = Bytes.toBytes("r2");
+  private static final byte[] row3 = Bytes.toBytes("r3");
+  private static final byte[] test = Bytes.toBytes("test");
+
+  @Rule
+  public TestName name = new TestName();
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, TestWALObserver.class.getName());
+    util = new HBaseTestingUtility(conf);
+    util.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  private void createTable(String coprocessor) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(tableName)
+        .addFamily(new HColumnDescriptor(dummy))
+        .addFamily(new HColumnDescriptor(test))
+        .addCoprocessor(coprocessor);
+    util.getHBaseAdmin().createTable(htd);
+  }
+
+  /**
+   * Test various multiput operations.
+   * @throws Exception
+   */
+  @Test
+  public void testMulti() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+  }
+
+  /**
+   * Tests that added mutations from coprocessors end up in the WAL.
+   */
+  @Test
+  public void testCPMutationsAreWrittenToWALEdit() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+
+    assertNotNull(TestWALObserver.savedEdit);
+    assertEquals(4, TestWALObserver.savedEdit.getCells().size());
+  }
+
+  private static void assertRowCount(Table t, int expected) throws IOException {
+    try (ResultScanner scanner = t.getScanner(new Scan())) {
+      int i = 0;
+      for (Result r: scanner) {
+        LOG.info(r.toString());
+        i++;
+      }
+      assertEquals(expected, i);
+    }
+  }
+
+  @Test
+  public void testDeleteCell() throws Exception {
+    createTable(TestDeleteCellCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteFamily() throws Exception {
+    createTable(TestDeleteFamilyCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addFamily(test)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteRow() throws Exception {
+    createTable(TestDeleteRowCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  public static class TestMultiMutationCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+      List<Cell> cells = mut.getFamilyCellMap().get(test);
+      Put[] puts = new Put[] {
+          new Put(row1).addColumn(test, dummy, cells.get(0).getTimestamp(),
+            Bytes.toBytes("cpdummy")),
+          new Put(row2).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+          new Put(row3).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+      };
+      LOG.info("Putting:" + puts);
+      miniBatchOp.addOperationsFromCP(0, puts);
+    }
+  }
+
+  public static class TestDeleteCellCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addColumns(test, dummy, cells.get(0).getTimestamp()),
+            new Delete(row2).addColumns(test, dummy, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteFamilyCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addFamily(test, cells.get(0).getTimestamp()),
+            new Delete(row2).addFamily(test, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteRowCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1, cells.get(0).getTimestamp()),
+            new Delete(row2, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestWALObserver extends BaseWALObserver {
+    static WALEdit savedEdit = null;
+    @Override
+    public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
+        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
+        savedEdit = logEdit;
+      }
+      super.postWALWrite(ctx, info, logKey, logEdit);
+    }
+  }
+}


[2/4] hbase git commit: HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)

Posted by en...@apache.org.
HBASE-15600 Add provision for adding mutations to memstore or able to write to same region in batchMutate coprocessor hooks (Rajeshbabu and Enis)

Conflicts:
	hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java


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

Branch: refs/heads/branch-1
Commit: 602b2839c688b96a57389bbfe20cd0e2073f22e6
Parents: 7fed7a8
Author: Enis Soztutar <en...@apache.org>
Authored: Tue Jun 7 20:12:51 2016 -0700
Committer: Enis Soztutar <en...@apache.org>
Committed: Tue Jun 7 20:32:06 2016 -0700

----------------------------------------------------------------------
 .../hadoop/hbase/regionserver/HRegion.java      |  80 ++++--
 .../MiniBatchOperationInProgress.java           |  29 +-
 .../regionserver/MultiRowMutationProcessor.java |   2 +-
 ...erverForAddingMutationsFromCoprocessors.java | 282 +++++++++++++++++++
 4 files changed, 373 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/602b2839/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 2e2ecfe..8eb16a6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -3079,18 +3079,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         }
 
         try {
-          if (isPutMutation) {
-            // Check the families in the put. If bad, skip this one.
-            if (isInReplay) {
-              removeNonExistentColumnFamilyForReplay(familyMap);
-            } else {
-              checkFamilies(familyMap.keySet());
-            }
-            checkTimestamps(mutation.getFamilyCellMap(), now);
-          } else {
-            prepareDelete((Delete) mutation);
-          }
-          checkRow(mutation.getRow(), "doMiniBatchMutation");
+          checkAndPrepareMutation(mutation, batchOp.isInReplay(), familyMap, now);
         } catch (NoSuchColumnFamilyException nscf) {
           LOG.warn("No such column family in batch mutation", nscf);
           batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
@@ -3198,7 +3187,38 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
           new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
           batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
-        if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
+        if (coprocessorHost.preBatchMutate(miniBatchOp)) {
+          return 0L;
+        } else {
+          for (int i = firstIndex; i < lastIndexExclusive; i++) {
+            if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
+              // lastIndexExclusive was incremented above.
+              continue;
+            }
+            // we pass (i - firstIndex) below since the call expects a relative index
+            Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - firstIndex);
+            if (cpMutations == null) {
+              continue;
+            }
+            // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
+            for (int j = 0; j < cpMutations.length; j++) {
+              Mutation cpMutation = cpMutations[j];
+              Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
+              checkAndPrepareMutation(cpMutation, isInReplay, cpFamilyMap, now);
+
+              // Acquire row locks. If not, the whole batch will fail.
+              acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true));
+
+              if (cpMutation.getDurability() == Durability.SKIP_WAL) {
+                recordMutationWithoutWal(cpFamilyMap);
+              }
+
+              // Returned mutations from coprocessor correspond to the Mutation at index i. We can
+              // directly add the cells from those mutations to the familyMaps of this mutation.
+              mergeFamilyMaps(familyMaps[i], cpFamilyMap); // will get added to the memstore later
+            }
+          }
+        }
       }
 
       // ------------------------------------
@@ -3422,9 +3442,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         // call the coprocessor hook to do any finalization steps
         // after the put is done
         MiniBatchOperationInProgress<Mutation> miniBatchOp =
-            new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
-                batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex,
-                lastIndexExclusive);
+          new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
+          batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
         coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
       }
 
@@ -3432,6 +3451,18 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
+      Map<byte[], List<Cell>> toBeMerged) {
+    for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
+      List<Cell> cells = familyMap.get(entry.getKey());
+      if (cells == null) {
+        familyMap.put(entry.getKey(), entry.getValue());
+      } else {
+        cells.addAll(entry.getValue());
+      }
+    }
+  }
+
   /**
    * Returns effective durability from the passed durability and
    * the table descriptor.
@@ -3861,6 +3892,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     }
   }
 
+  private void checkAndPrepareMutation(Mutation mutation, boolean replay,
+      final Map<byte[], List<Cell>> familyMap, final long now)
+          throws IOException {
+    if (mutation instanceof Put) {
+      // Check the families in the put. If bad, skip this one.
+      if (replay) {
+        removeNonExistentColumnFamilyForReplay(familyMap);
+      } else {
+        checkFamilies(familyMap.keySet());
+      }
+      checkTimestamps(mutation.getFamilyCellMap(), now);
+    } else {
+      prepareDelete((Delete)mutation);
+    }
+    checkRow(mutation.getRow(), "doMiniBatchMutation");
+  }
+
   /**
    * During replay, there could exist column families which are removed between region server
    * failure and replay

http://git-wip-us.apache.org/repos/asf/hbase/blob/602b2839/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
index 2b12dec..cdbecac 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MiniBatchOperationInProgress.java
@@ -18,20 +18,22 @@
 package org.apache.hadoop.hbase.regionserver;
 
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
+import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
 
 /**
  * Wraps together the mutations which are applied as a batch to the region and their operation
- * status and WALEdits. 
+ * status and WALEdits.
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#preBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @see org.apache.hadoop.hbase.coprocessor.RegionObserver#postBatchMutate(
  * ObserverContext, MiniBatchOperationInProgress)
  * @param T Pair&lt;Mutation, Integer&gt; pair of Mutations and associated rowlock ids .
  */
-@InterfaceAudience.Private
+@InterfaceAudience.LimitedPrivate("Coprocessors")
 public class MiniBatchOperationInProgress<T> {
   private final T[] operations;
+  private Mutation[][] operationsFromCoprocessors;
   private final OperationStatus[] retCodeDetails;
   private final WALEdit[] walEditsFromCoprocessors;
   private final int firstIndex;
@@ -63,7 +65,7 @@ public class MiniBatchOperationInProgress<T> {
 
   /**
    * Sets the status code for the operation(Mutation) at the specified position.
-   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver} 
+   * By setting this status, {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
    * can make HRegion to skip Mutations.
    * @param index
    * @param opStatus
@@ -103,4 +105,25 @@ public class MiniBatchOperationInProgress<T> {
     }
     return this.firstIndex + index;
   }
+
+  /**
+   * Add more Mutations corresponding to the Mutation at the given index to be committed atomically
+   * in the same batch. These mutations are applied to the WAL and applied to the memstore as well.
+   * The timestamp of the cells in the given Mutations MUST be obtained from the original mutation.
+   *
+   * @param index the index that corresponds to the original mutation index in the batch
+   * @param newOperations the Mutations to add
+   */
+  public void addOperationsFromCP(int index, Mutation[] newOperations) {
+    if (this.operationsFromCoprocessors == null) {
+      // lazy allocation to save on object allocation in case this is not used
+      this.operationsFromCoprocessors = new Mutation[operations.length][];
+    }
+    this.operationsFromCoprocessors[getAbsoluteIndex(index)] = newOperations;
+  }
+
+  public Mutation[] getOperationsFromCoprocessors(int index) {
+    return operationsFromCoprocessors == null ? null :
+        operationsFromCoprocessors[getAbsoluteIndex(index)];
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/602b2839/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
index 1947a1b..995ea93 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MultiRowMutationProcessor.java
@@ -60,7 +60,7 @@ MultiRowMutationProcessorResponse> {
   public boolean readOnly() {
     return false;
   }
-  
+
   @Override
   public MultiRowMutationProcessorResponse getResult() {
     return MultiRowMutationProcessorResponse.getDefaultInstance();

http://git-wip-us.apache.org/repos/asf/hbase/blob/602b2839/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
new file mode 100644
index 0000000..0663b0b
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverForAddingMutationsFromCoprocessors.java
@@ -0,0 +1,282 @@
+/**
+ * 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.coprocessor;
+
+import static org.junit.Assert.*;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.regionserver.MiniBatchOperationInProgress;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.wal.WALKey;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+
+@Category(MediumTests.class)
+public class TestRegionObserverForAddingMutationsFromCoprocessors {
+
+  private static final Log LOG
+    = LogFactory.getLog(TestRegionObserverForAddingMutationsFromCoprocessors.class);
+
+  private static HBaseTestingUtility util;
+  private static final byte[] dummy = Bytes.toBytes("dummy");
+  private static final byte[] row1 = Bytes.toBytes("r1");
+  private static final byte[] row2 = Bytes.toBytes("r2");
+  private static final byte[] row3 = Bytes.toBytes("r3");
+  private static final byte[] test = Bytes.toBytes("test");
+
+  @Rule
+  public TestName name = new TestName();
+  private TableName tableName;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    Configuration conf = HBaseConfiguration.create();
+    conf.set(CoprocessorHost.WAL_COPROCESSOR_CONF_KEY, TestWALObserver.class.getName());
+    util = new HBaseTestingUtility(conf);
+    util.startMiniCluster();
+  }
+
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    util.shutdownMiniCluster();
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    tableName = TableName.valueOf(name.getMethodName());
+  }
+
+  private void createTable(String coprocessor) throws IOException {
+    HTableDescriptor htd = new HTableDescriptor(tableName)
+        .addFamily(new HColumnDescriptor(dummy))
+        .addFamily(new HColumnDescriptor(test))
+        .addCoprocessor(coprocessor);
+    util.getHBaseAdmin().createTable(htd);
+  }
+
+  /**
+   * Test various multiput operations.
+   * @throws Exception
+   */
+  @Test
+  public void testMulti() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+  }
+
+  /**
+   * Tests that added mutations from coprocessors end up in the WAL.
+   */
+  @Test
+  public void testCPMutationsAreWrittenToWALEdit() throws Exception {
+    createTable(TestMultiMutationCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(new Put(row1).addColumn(test, dummy, dummy));
+      assertRowCount(t, 3);
+    }
+
+    assertNotNull(TestWALObserver.savedEdit);
+    assertEquals(4, TestWALObserver.savedEdit.getCells().size());
+  }
+
+  private static void assertRowCount(Table t, int expected) throws IOException {
+    try (ResultScanner scanner = t.getScanner(new Scan())) {
+      int i = 0;
+      for (Result r: scanner) {
+        LOG.info(r.toString());
+        i++;
+      }
+      assertEquals(expected, i);
+    }
+  }
+
+  @Test
+  public void testDeleteCell() throws Exception {
+    createTable(TestDeleteCellCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteFamily() throws Exception {
+    createTable(TestDeleteFamilyCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addFamily(test)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  @Test
+  public void testDeleteRow() throws Exception {
+    createTable(TestDeleteRowCoprocessor.class.getName());
+
+    try (Table t = util.getConnection().getTable(tableName)) {
+      t.put(Lists.newArrayList(
+        new Put(row1).addColumn(test, dummy, dummy),
+        new Put(row2).addColumn(test, dummy, dummy),
+        new Put(row3).addColumn(test, dummy, dummy)
+          ));
+
+      assertRowCount(t, 3);
+
+      t.delete(new Delete(test).addColumn(test, dummy)); // delete non-existing row
+      assertRowCount(t, 1);
+    }
+  }
+
+  public static class TestMultiMutationCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+      List<Cell> cells = mut.getFamilyCellMap().get(test);
+      Put[] puts = new Put[] {
+          new Put(row1).addColumn(test, dummy, cells.get(0).getTimestamp(),
+            Bytes.toBytes("cpdummy")),
+          new Put(row2).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+          new Put(row3).addColumn(test, dummy, cells.get(0).getTimestamp(), dummy),
+      };
+      LOG.info("Putting:" + puts);
+      miniBatchOp.addOperationsFromCP(0, puts);
+    }
+  }
+
+  public static class TestDeleteCellCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addColumns(test, dummy, cells.get(0).getTimestamp()),
+            new Delete(row2).addColumns(test, dummy, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteFamilyCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1).addFamily(test, cells.get(0).getTimestamp()),
+            new Delete(row2).addFamily(test, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestDeleteRowCoprocessor extends BaseRegionObserver {
+    @Override
+    public void preBatchMutate(ObserverContext<RegionCoprocessorEnvironment> c,
+        MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
+      Mutation mut = miniBatchOp.getOperation(0);
+
+      if (mut instanceof Delete) {
+        List<Cell> cells = mut.getFamilyCellMap().get(test);
+        Delete[] deletes = new Delete[] {
+            // delete only 2 rows
+            new Delete(row1, cells.get(0).getTimestamp()),
+            new Delete(row2, cells.get(0).getTimestamp()),
+        };
+        LOG.info("Deleting:" + Arrays.toString(deletes));
+        miniBatchOp.addOperationsFromCP(0, deletes);
+      }
+    }
+  }
+
+  public static class TestWALObserver extends BaseWALObserver {
+    static WALEdit savedEdit = null;
+    @Override
+    public void postWALWrite(ObserverContext<? extends WALCoprocessorEnvironment> ctx,
+        HRegionInfo info, WALKey logKey, WALEdit logEdit) throws IOException {
+      if (info.getTable().equals(TableName.valueOf("testCPMutationsAreWrittenToWALEdit"))) {
+        savedEdit = logEdit;
+      }
+      super.postWALWrite(ctx, info, logKey, logEdit);
+    }
+  }
+}