You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2017/07/01 01:50:15 UTC

[1/2] hbase git commit: HBASE-18023 Log multi-* requests for more than threshold number of rows

Repository: hbase
Updated Branches:
  refs/heads/branch-1 3b28f66bf -> 743f454d2
  refs/heads/branch-1.3 8d40b2d98 -> 70386cbe4


HBASE-18023 Log multi-* requests for more than threshold number of rows

Signed-off-by: Josh Elser <el...@apache.org>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1
Commit: 743f454d25f70a06d6ba212e0bae648616474f8d
Parents: 3b28f66
Author: David Harju <dh...@salesforce.com>
Authored: Thu Jun 22 13:29:34 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Jun 30 18:45:36 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        |   7 +
 .../hbase/regionserver/RSRpcServices.java       |  52 ++++++-
 .../regionserver/TestMultiLogThreshold.java     | 143 +++++++++++++++++++
 3 files changed, 201 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/743f454d/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 84c5be3..3c4044d 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1597,4 +1597,11 @@ possible configurations would overwhelm and obscure the important.
        Timeout for regionservers to keep threads in snapshot request pool waiting
     </description>
    </property>
+   <property>
+    <name>hbase.rpc.rows.warning.threshold</name>
+    <value>1000</value>
+    <description>
+      Number of rows in a batch operation above which a warning will be logged.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/743f454d/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 7b81a99..68fb1fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -226,6 +226,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA = 10;
 
+  /**
+   * Number of rows in a batch operation above which a warning will be logged.
+   */
+  static final String BATCH_ROWS_THRESHOLD_NAME = "hbase.rpc.rows.warning.threshold";
+  /**
+   * Default value of {@link RSRpcServices#BATCH_ROWS_THRESHOLD_NAME}
+   */
+  static final int BATCH_ROWS_THRESHOLD_DEFAULT = 1000;
+
   // Request counter. (Includes requests that are not serviced by regions.)
   final Counter requestCount = new Counter();
 
@@ -273,6 +282,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   private final long minimumScanTimeLimitDelta;
 
   /**
+   * Row size threshold for multi requests above which a warning is logged
+   */
+  private final int rowSizeWarnThreshold;
+
+  /**
    * Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
    */
   private static final class RegionScannerHolder {
@@ -1005,9 +1019,33 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  // Exposed for testing
+  static interface LogDelegate {
+    void logBatchWarning(int sum, int rowSizeWarnThreshold);
+  }
+
+  private static LogDelegate DEFAULT_LOG_DELEGATE = new LogDelegate() {
+    @Override
+    public void logBatchWarning(int sum, int rowSizeWarnThreshold) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Large batch operation detected (greater than " + rowSizeWarnThreshold
+            + ") (HBASE-18023)." + " Requested Number of Rows: " + sum + " Client: "
+            + RpcServer.getRequestUserName() + "/" + RpcServer.getRemoteAddress());
+      }
+    }
+  };
+
+  private final LogDelegate ld;
+
   public RSRpcServices(HRegionServer rs) throws IOException {
-    regionServer = rs;
+    this(rs, DEFAULT_LOG_DELEGATE);
+  }
 
+  // Directly invoked only for testing
+  RSRpcServices(HRegionServer rs, LogDelegate ld) throws IOException {
+    this.ld = ld;
+    regionServer = rs;
+    rowSizeWarnThreshold = rs.conf.getInt(BATCH_ROWS_THRESHOLD_NAME, BATCH_ROWS_THRESHOLD_DEFAULT);
     RpcSchedulerFactory rpcSchedulerFactory;
     try {
       Class<?> rpcSchedulerFactoryClass = rs.conf.getClass(
@@ -2222,6 +2260,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  private void checkBatchSizeAndLogLargeSize(MultiRequest request) {
+    int sum = 0;
+    for (RegionAction regionAction : request.getRegionActionList()) {
+      sum += regionAction.getActionCount();
+    }
+    if (sum > rowSizeWarnThreshold) {
+      ld.logBatchWarning(sum, rowSizeWarnThreshold);
+    }
+  }
+
   /**
    * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
    *
@@ -2238,6 +2286,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new ServiceException(ie);
     }
 
+    checkBatchSizeAndLogLargeSize(request);
+
     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
     // It is also the conduit via which we pass back data.
     HBaseRpcController controller = (HBaseRpcController)rpcc;

http://git-wip-us.apache.org/repos/asf/hbase/blob/743f454d/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
new file mode 100644
index 0000000..23ab1ff
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
@@ -0,0 +1,143 @@
+/**
+ *
+ * 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.regionserver;
+
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Tests logging of large batch commands via Multi. Tests are fast, but uses a mini-cluster (to test
+ * via "Multi" commands) so classified as MediumTests
+ */
+@Category(MediumTests.class)
+public class TestMultiLogThreshold {
+
+  private static RSRpcServices SERVICES;
+
+  private static HBaseTestingUtility TEST_UTIL;
+  private static Configuration CONF;
+  private static final byte[] TEST_FAM = Bytes.toBytes("fam");
+  private static RSRpcServices.LogDelegate LD;
+  private static HRegionServer RS;
+  private static int THRESHOLD;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    final TableName tableName = TableName.valueOf("tableName");
+    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
+    CONF = TEST_UTIL.getConfiguration();
+    THRESHOLD = CONF.getInt(RSRpcServices.BATCH_ROWS_THRESHOLD_NAME,
+      RSRpcServices.BATCH_ROWS_THRESHOLD_DEFAULT);
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL.createTable(tableName, TEST_FAM);
+    RS = TEST_UTIL.getRSForFirstRegionInTable(tableName);
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+    LD = Mockito.mock(RSRpcServices.LogDelegate.class);
+    SERVICES = new RSRpcServices(RS, LD);
+  }
+
+  private enum ActionType {
+    REGION_ACTIONS, ACTIONS;
+  }
+
+  /**
+   * Sends a multi request with a certain amount of rows, will populate Multi command with either
+   * "rows" number of RegionActions with one Action each or one RegionAction with "rows" number of
+   * Actions
+   */
+  private void sendMultiRequest(int rows, ActionType actionType) throws ServiceException {
+    RpcController rpcc = Mockito.mock(RpcController.class);
+    MultiRequest.Builder builder = MultiRequest.newBuilder();
+    int numRAs = 1;
+    int numAs = 1;
+    switch (actionType) {
+    case REGION_ACTIONS:
+      numRAs = rows;
+      break;
+    case ACTIONS:
+      numAs = rows;
+      break;
+    }
+    for (int i = 0; i < numRAs; i++) {
+      RegionAction.Builder rab = RegionAction.newBuilder();
+      rab.setRegion(RequestConverter.buildRegionSpecifier(
+        HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME,
+        new String("someStuff" + i).getBytes()));
+      for (int j = 0; j < numAs; j++) {
+        Action.Builder ab = Action.newBuilder();
+        rab.addAction(ab.build());
+      }
+      builder.addRegionAction(rab.build());
+    }
+    try {
+      SERVICES.multi(rpcc, builder.build());
+    } catch (ClassCastException e) {
+      // swallow expected exception due to mocked RpcController
+    }
+  }
+
+  @Test
+  public void testMultiLogThresholdRegionActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD + 1, ActionType.REGION_ACTIONS);
+    verify(LD, Mockito.times(1)).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiNoLogThresholdRegionActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD, ActionType.REGION_ACTIONS);
+    verify(LD, Mockito.never()).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiLogThresholdActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD + 1, ActionType.ACTIONS);
+    verify(LD, Mockito.times(1)).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiNoLogThresholdAction() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD, ActionType.ACTIONS);
+    verify(LD, Mockito.never()).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+}


[2/2] hbase git commit: HBASE-18023 Log multi-* requests for more than threshold number of rows

Posted by ap...@apache.org.
HBASE-18023 Log multi-* requests for more than threshold number of rows

Signed-off-by: Josh Elser <el...@apache.org>
Amending-Author: Andrew Purtell <ap...@apache.org>


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

Branch: refs/heads/branch-1.3
Commit: 70386cbe4cd905b50bec5fa67a751141ea89ed9f
Parents: 8d40b2d
Author: David Harju <dh...@salesforce.com>
Authored: Thu Jun 22 13:29:34 2017 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Jun 30 18:46:07 2017 -0700

----------------------------------------------------------------------
 .../src/main/resources/hbase-default.xml        |   7 +
 .../hbase/regionserver/RSRpcServices.java       |  52 ++++++-
 .../regionserver/TestMultiLogThreshold.java     | 143 +++++++++++++++++++
 3 files changed, 201 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/70386cbe/hbase-common/src/main/resources/hbase-default.xml
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml
index 6f2134e..6a20895 100644
--- a/hbase-common/src/main/resources/hbase-default.xml
+++ b/hbase-common/src/main/resources/hbase-default.xml
@@ -1543,4 +1543,11 @@ possible configurations would overwhelm and obscure the important.
        Timeout for regionservers to keep threads in snapshot request pool waiting
     </description>
    </property>
+   <property>
+    <name>hbase.rpc.rows.warning.threshold</name>
+    <value>1000</value>
+    <description>
+      Number of rows in a batch operation above which a warning will be logged.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/hbase/blob/70386cbe/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index 23b347a..47e50c7 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -225,6 +225,15 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
    */
   private static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA = 10;
 
+  /**
+   * Number of rows in a batch operation above which a warning will be logged.
+   */
+  static final String BATCH_ROWS_THRESHOLD_NAME = "hbase.rpc.rows.warning.threshold";
+  /**
+   * Default value of {@link RSRpcServices#BATCH_ROWS_THRESHOLD_NAME}
+   */
+  static final int BATCH_ROWS_THRESHOLD_DEFAULT = 1000;
+
   // Request counter. (Includes requests that are not serviced by regions.)
   final Counter requestCount = new Counter();
 
@@ -272,6 +281,11 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
   private final long minimumScanTimeLimitDelta;
 
   /**
+   * Row size threshold for multi requests above which a warning is logged
+   */
+  private final int rowSizeWarnThreshold;
+
+  /**
    * Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
    */
   private static final class RegionScannerHolder {
@@ -957,9 +971,33 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  // Exposed for testing
+  static interface LogDelegate {
+    void logBatchWarning(int sum, int rowSizeWarnThreshold);
+  }
+
+  private static LogDelegate DEFAULT_LOG_DELEGATE = new LogDelegate() {
+    @Override
+    public void logBatchWarning(int sum, int rowSizeWarnThreshold) {
+      if (LOG.isWarnEnabled()) {
+        LOG.warn("Large batch operation detected (greater than " + rowSizeWarnThreshold
+            + ") (HBASE-18023)." + " Requested Number of Rows: " + sum + " Client: "
+            + RpcServer.getRequestUserName() + "/" + RpcServer.getRemoteAddress());
+      }
+    }
+  };
+
+  private final LogDelegate ld;
+
   public RSRpcServices(HRegionServer rs) throws IOException {
-    regionServer = rs;
+    this(rs, DEFAULT_LOG_DELEGATE);
+  }
 
+  // Directly invoked only for testing
+  RSRpcServices(HRegionServer rs, LogDelegate ld) throws IOException {
+    this.ld = ld;
+    regionServer = rs;
+    rowSizeWarnThreshold = rs.conf.getInt(BATCH_ROWS_THRESHOLD_NAME, BATCH_ROWS_THRESHOLD_DEFAULT);
     RpcSchedulerFactory rpcSchedulerFactory;
     try {
       Class<?> rpcSchedulerFactoryClass = rs.conf.getClass(
@@ -2162,6 +2200,16 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
     }
   }
 
+  private void checkBatchSizeAndLogLargeSize(MultiRequest request) {
+    int sum = 0;
+    for (RegionAction regionAction : request.getRegionActionList()) {
+      sum += regionAction.getActionCount();
+    }
+    if (sum > rowSizeWarnThreshold) {
+      ld.logBatchWarning(sum, rowSizeWarnThreshold);
+    }
+  }
+
   /**
    * Execute multiple actions on a table: get, mutate, and/or execCoprocessor
    *
@@ -2178,6 +2226,8 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
       throw new ServiceException(ie);
     }
 
+    checkBatchSizeAndLogLargeSize(request);
+
     // rpc controller is how we bring in data via the back door;  it is unprotobuf'ed data.
     // It is also the conduit via which we pass back data.
     PayloadCarryingRpcController controller = (PayloadCarryingRpcController)rpcc;

http://git-wip-us.apache.org/repos/asf/hbase/blob/70386cbe/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
new file mode 100644
index 0000000..23ab1ff
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestMultiLogThreshold.java
@@ -0,0 +1,143 @@
+/**
+ *
+ * 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.regionserver;
+
+import static org.mockito.Mockito.verify;
+
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.protobuf.RequestConverter;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Action;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MultiRequest;
+import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.RegionAction;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.util.Bytes;
+
+/**
+ * Tests logging of large batch commands via Multi. Tests are fast, but uses a mini-cluster (to test
+ * via "Multi" commands) so classified as MediumTests
+ */
+@Category(MediumTests.class)
+public class TestMultiLogThreshold {
+
+  private static RSRpcServices SERVICES;
+
+  private static HBaseTestingUtility TEST_UTIL;
+  private static Configuration CONF;
+  private static final byte[] TEST_FAM = Bytes.toBytes("fam");
+  private static RSRpcServices.LogDelegate LD;
+  private static HRegionServer RS;
+  private static int THRESHOLD;
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    final TableName tableName = TableName.valueOf("tableName");
+    TEST_UTIL = HBaseTestingUtility.createLocalHTU();
+    CONF = TEST_UTIL.getConfiguration();
+    THRESHOLD = CONF.getInt(RSRpcServices.BATCH_ROWS_THRESHOLD_NAME,
+      RSRpcServices.BATCH_ROWS_THRESHOLD_DEFAULT);
+    TEST_UTIL.startMiniCluster();
+    TEST_UTIL.createTable(tableName, TEST_FAM);
+    RS = TEST_UTIL.getRSForFirstRegionInTable(tableName);
+  }
+
+  @Before
+  public void setupTest() throws Exception {
+    LD = Mockito.mock(RSRpcServices.LogDelegate.class);
+    SERVICES = new RSRpcServices(RS, LD);
+  }
+
+  private enum ActionType {
+    REGION_ACTIONS, ACTIONS;
+  }
+
+  /**
+   * Sends a multi request with a certain amount of rows, will populate Multi command with either
+   * "rows" number of RegionActions with one Action each or one RegionAction with "rows" number of
+   * Actions
+   */
+  private void sendMultiRequest(int rows, ActionType actionType) throws ServiceException {
+    RpcController rpcc = Mockito.mock(RpcController.class);
+    MultiRequest.Builder builder = MultiRequest.newBuilder();
+    int numRAs = 1;
+    int numAs = 1;
+    switch (actionType) {
+    case REGION_ACTIONS:
+      numRAs = rows;
+      break;
+    case ACTIONS:
+      numAs = rows;
+      break;
+    }
+    for (int i = 0; i < numRAs; i++) {
+      RegionAction.Builder rab = RegionAction.newBuilder();
+      rab.setRegion(RequestConverter.buildRegionSpecifier(
+        HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME,
+        new String("someStuff" + i).getBytes()));
+      for (int j = 0; j < numAs; j++) {
+        Action.Builder ab = Action.newBuilder();
+        rab.addAction(ab.build());
+      }
+      builder.addRegionAction(rab.build());
+    }
+    try {
+      SERVICES.multi(rpcc, builder.build());
+    } catch (ClassCastException e) {
+      // swallow expected exception due to mocked RpcController
+    }
+  }
+
+  @Test
+  public void testMultiLogThresholdRegionActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD + 1, ActionType.REGION_ACTIONS);
+    verify(LD, Mockito.times(1)).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiNoLogThresholdRegionActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD, ActionType.REGION_ACTIONS);
+    verify(LD, Mockito.never()).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiLogThresholdActions() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD + 1, ActionType.ACTIONS);
+    verify(LD, Mockito.times(1)).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+  @Test
+  public void testMultiNoLogThresholdAction() throws ServiceException, IOException {
+    sendMultiRequest(THRESHOLD, ActionType.ACTIONS);
+    verify(LD, Mockito.never()).logBatchWarning(Mockito.anyInt(), Mockito.anyInt());
+  }
+
+}