From commits-return-88930-archive-asf-public=cust-asf.ponee.io@hbase.apache.org Sun Sep 1 14:14:25 2019 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by mx-eu-01.ponee.io (Postfix) with SMTP id 6FA84180668 for ; Sun, 1 Sep 2019 16:14:25 +0200 (CEST) Received: (qmail 9775 invoked by uid 500); 1 Sep 2019 14:14:24 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 9619 invoked by uid 99); 1 Sep 2019 14:14:23 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 01 Sep 2019 14:14:23 +0000 Received: by gitbox.apache.org (ASF Mail Server at gitbox.apache.org, from userid 33) id 0EE1C8614B; Sun, 1 Sep 2019 14:14:22 +0000 (UTC) Date: Sun, 01 Sep 2019 14:14:24 +0000 To: "commits@hbase.apache.org" Subject: [hbase] 02/07: HBASE-22662 Move RSGroupInfoManager to hbase-server (#368) MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 8bit From: zhangduo@apache.org In-Reply-To: <156734726224.11944.11734111761583364770@gitbox.apache.org> References: <156734726224.11944.11734111761583364770@gitbox.apache.org> X-Git-Host: gitbox.apache.org X-Git-Repo: hbase X-Git-Refname: refs/heads/HBASE-22514 X-Git-Reftype: branch X-Git-Rev: 63f3d1a9f39585f315f01e9dc6b5fe9d91b76070 X-Git-NotificationType: diff X-Git-Multimail-Version: 1.5.dev Auto-Submitted: auto-generated Message-Id: <20190901141423.0EE1C8614B@gitbox.apache.org> This is an automated email from the ASF dual-hosted git repository. zhangduo pushed a commit to branch HBASE-22514 in repository https://gitbox.apache.org/repos/asf/hbase.git commit 63f3d1a9f39585f315f01e9dc6b5fe9d91b76070 Author: Duo Zhang AuthorDate: Thu Jul 11 10:34:05 2019 +0800 HBASE-22662 Move RSGroupInfoManager to hbase-server (#368) Signed-off-by: Guanghao Zhang --- hbase-rsgroup/pom.xml | 4 - .../hbase/rsgroup/RSGroupBasedLoadBalancer.java | 4 +- .../hbase/rsgroup/TestRSGroupsOfflineMode.java | 6 +- .../hbase/rsgroup/VerifyingRSGroupAdminClient.java | 6 +- .../hadoop/hbase/rsgroup/RSGroupInfoManager.java | 26 +---- .../hbase/rsgroup/RSGroupInfoManagerImpl.java | 108 +++++++++++---------- 6 files changed, 71 insertions(+), 83 deletions(-) diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml index 05cbf34..7b52af7 100644 --- a/hbase-rsgroup/pom.xml +++ b/hbase-rsgroup/pom.xml @@ -107,10 +107,6 @@ hbase-shaded-miscellaneous - com.google.protobuf - protobuf-java - - org.apache.zookeeper zookeeper diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java index 9709fb5..76c96f5 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java +++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupBasedLoadBalancer.java @@ -121,8 +121,8 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer { public List balanceCluster(Map> clusterState) throws HBaseIOException { if (!isOnline()) { - throw new ConstraintException(RSGroupInfoManager.RSGROUP_TABLE_NAME + - " is not online, unable to perform balance"); + throw new ConstraintException( + RSGroupInfoManager.class.getSimpleName() + " is not online, unable to perform balance"); } // Calculate correct assignments and a list of RegionPlan for mis-placed regions diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java index 39cf164..60887e4 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/TestRSGroupsOfflineMode.java @@ -140,7 +140,7 @@ public class TestRSGroupsOfflineMode { } }); // Move table to group and wait. - groupAdmin.moveTables(Sets.newHashSet(RSGroupInfoManager.RSGROUP_TABLE_NAME), newGroup); + groupAdmin.moveTables(Sets.newHashSet(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME), newGroup); LOG.info("Waiting for move table..."); TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate() { @Override @@ -169,7 +169,7 @@ public class TestRSGroupsOfflineMode { // Make sure balancer is in offline mode, since this is what we're testing. assertFalse(groupMgr.isOnline()); // Verify the group affiliation that's loaded from ZK instead of tables. - assertEquals(newGroup, groupMgr.getRSGroupOfTable(RSGroupInfoManager.RSGROUP_TABLE_NAME)); + assertEquals(newGroup, groupMgr.getRSGroupOfTable(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME)); assertEquals(RSGroupInfo.DEFAULT_GROUP, groupMgr.getRSGroupOfTable(failoverTable)); // Kill final regionserver to see the failover happens for all tables except GROUP table since // it's group does not have any online RS. @@ -182,7 +182,7 @@ public class TestRSGroupsOfflineMode { return failoverRS.getRegions(failoverTable).size() >= 1; } }); - Assert.assertEquals(0, failoverRS.getRegions(RSGroupInfoManager.RSGROUP_TABLE_NAME).size()); + Assert.assertEquals(0, failoverRS.getRegions(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME).size()); // Need this for minicluster to shutdown cleanly. master.stopMaster(); diff --git a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java index 2ad30e4..fcaf1a7 100644 --- a/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java +++ b/hbase-rsgroup/src/test/java/org/apache/hadoop/hbase/rsgroup/VerifyingRSGroupAdminClient.java @@ -52,7 +52,7 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin { throws IOException { wrapped = RSGroupAdmin; table = ConnectionFactory.createConnection(conf) - .getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME); + .getTable(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME); zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null); } @@ -126,8 +126,8 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin { RSGroupProtos.RSGroupInfo proto = RSGroupProtos.RSGroupInfo.parseFrom( result.getValue( - RSGroupInfoManager.META_FAMILY_BYTES, - RSGroupInfoManager.META_QUALIFIER_BYTES)); + RSGroupInfoManagerImpl.META_FAMILY_BYTES, + RSGroupInfoManagerImpl.META_QUALIFIER_BYTES)); groupMap.put(proto.getName(), ProtobufUtil.toGroupInfo(proto)); } Assert.assertEquals(Sets.newHashSet(groupMap.values()), diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java similarity index 82% rename from hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java index 398e8a4..70aeabf 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManager.java @@ -15,38 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.hadoop.hbase.rsgroup; import java.io.IOException; import java.util.List; import java.util.Set; - -import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.net.Address; -import org.apache.hadoop.hbase.util.Bytes; import org.apache.yetus.audience.InterfaceAudience; /** - * Interface used to manage RSGroupInfo storage. An implementation - * has the option to support offline mode. - * See {@link RSGroupBasedLoadBalancer} + * Interface used to manage RSGroupInfo storage. An implementation has the option to support offline + * mode. See {@code RSGroupBasedLoadBalancer}. */ @InterfaceAudience.Private public interface RSGroupInfoManager { - String REASSIGN_WAIT_INTERVAL_KEY = "hbase.rsgroup.reassign.wait"; - long DEFAULT_REASSIGN_WAIT_INTERVAL = 30 * 1000L; - - //Assigned before user tables - TableName RSGROUP_TABLE_NAME = - TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup"); - String rsGroupZNode = "rsgroup"; - byte[] META_FAMILY_BYTES = Bytes.toBytes("m"); - byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i"); - byte[] ROW_KEY = {0}; - void start(); /** @@ -86,7 +70,6 @@ public interface RSGroupInfoManager { /** * Set the group membership of a set of tables - * * @param tableNames set of tables to move * @param groupName name of group of tables to move to */ @@ -104,7 +87,6 @@ public interface RSGroupInfoManager { /** * Whether the manager is able to fully return group metadata - * * @return whether the manager is in online mode */ boolean isOnline(); @@ -116,8 +98,8 @@ public interface RSGroupInfoManager { * @param srcGroup groupName being moved from * @param dstGroup groupName being moved to */ - void moveServersAndTables(Set
servers, Set tables, - String srcGroup, String dstGroup) throws IOException; + void moveServersAndTables(Set
servers, Set tables, String srcGroup, + String dstGroup) throws IOException; /** * Remove decommissioned servers from rsgroup diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java similarity index 90% rename from hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java index b54f088..8aa7520 100644 --- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.hbase.rsgroup; -import com.google.protobuf.ServiceException; import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; @@ -34,10 +33,12 @@ import java.util.SortedSet; import java.util.TreeSet; import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.DoNotRetryIOException; +import org.apache.hadoop.hbase.NamespaceDescriptor; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.AsyncClusterConnection; +import org.apache.hadoop.hbase.client.AsyncTable; import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder; -import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Get; @@ -45,14 +46,11 @@ 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.client.TableDescriptor; import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.exceptions.DeserializationException; -import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.ServerListener; import org.apache.hadoop.hbase.master.TableStateManager; @@ -62,10 +60,14 @@ import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.procedure2.Procedure; import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; +import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; +import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest; +import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsResponse; import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FutureUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKWatcher; @@ -75,6 +77,7 @@ import org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hbase.thirdparty.com.google.common.collect.Lists; import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Sets; @@ -87,13 +90,13 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; * RSGroupInfo Map at {@link #rsGroupMap} and a Map of tables to the name of the rsgroup they belong * too (in {@link #tableMap}). These Maps are persisted to the hbase:rsgroup table (and cached in * zk) on each modification. - *

+ *

* Mutations on state are synchronized but reads can continue without having to wait on an instance * monitor, mutations do wholesale replace of the Maps on update -- Copy-On-Write; the local Maps of * state are read-only, just-in-case (see flushConfig). - *

+ *

* Reads must not block else there is a danger we'll deadlock. - *

+ *

* Clients of this class, the {@link RSGroupAdminEndpoint} for example, want to query and then act * on the results of the query modifying cache in zookeeper without another thread making * intermediate modifications. These clients synchronize on the 'this' instance so no other has @@ -103,6 +106,24 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Sets; final class RSGroupInfoManagerImpl implements RSGroupInfoManager { private static final Logger LOG = LoggerFactory.getLogger(RSGroupInfoManagerImpl.class); + private static final String REASSIGN_WAIT_INTERVAL_KEY = "hbase.rsgroup.reassign.wait"; + private static final long DEFAULT_REASSIGN_WAIT_INTERVAL = 30 * 1000L; + + // Assigned before user tables + @VisibleForTesting + static final TableName RSGROUP_TABLE_NAME = + TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "rsgroup"); + + private static final String RS_GROUP_ZNODE = "rsgroup"; + + @VisibleForTesting + static final byte[] META_FAMILY_BYTES = Bytes.toBytes("m"); + + @VisibleForTesting + static final byte[] META_QUALIFIER_BYTES = Bytes.toBytes("i"); + + private static final byte[] ROW_KEY = { 0 }; + /** Table descriptor for hbase:rsgroup catalog table */ private static final TableDescriptor RSGROUP_TABLE_DESC; static { @@ -125,7 +146,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { private volatile Map tableMap = Collections.emptyMap(); private final MasterServices masterServices; - private final Connection conn; + private final AsyncClusterConnection conn; private final ZKWatcher watcher; private final RSGroupStartupWorker rsGroupStartupWorker; // contains list of groups that were last flushed to persistent store @@ -136,7 +157,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { private RSGroupInfoManagerImpl(MasterServices masterServices) throws IOException { this.masterServices = masterServices; this.watcher = masterServices.getZooKeeper(); - this.conn = masterServices.getConnection(); + this.conn = masterServices.getAsyncClusterConnection(); this.rsGroupStartupWorker = new RSGroupStartupWorker(); } @@ -349,25 +370,25 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { } } - List retrieveGroupListFromGroupTable() throws IOException { + private List retrieveGroupListFromGroupTable() throws IOException { List rsGroupInfoList = Lists.newArrayList(); - try (Table table = conn.getTable(RSGROUP_TABLE_NAME); - ResultScanner scanner = table.getScanner(new Scan())) { + AsyncTable table = conn.getTable(RSGROUP_TABLE_NAME); + try (ResultScanner scanner = table.getScanner(META_FAMILY_BYTES, META_QUALIFIER_BYTES)) { for (Result result;;) { result = scanner.next(); if (result == null) { break; } RSGroupProtos.RSGroupInfo proto = RSGroupProtos.RSGroupInfo - .parseFrom(result.getValue(META_FAMILY_BYTES, META_QUALIFIER_BYTES)); + .parseFrom(result.getValue(META_FAMILY_BYTES, META_QUALIFIER_BYTES)); rsGroupInfoList.add(ProtobufUtil.toGroupInfo(proto)); } } return rsGroupInfoList; } - List retrieveGroupListFromZookeeper() throws IOException { - String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, rsGroupZNode); + private List retrieveGroupListFromZookeeper() throws IOException { + String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, RS_GROUP_ZNODE); List RSGroupInfoList = Lists.newArrayList(); // Overwrite any info stored by table, this takes precedence try { @@ -519,7 +540,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { resetRSGroupAndTableMaps(newGroupMap, newTableMap); try { - String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, rsGroupZNode); + String groupBasePath = + ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, RS_GROUP_ZNODE); ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC); List zkOps = new ArrayList<>(newGroupMap.size()); @@ -702,11 +724,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { createRSGroupTable(); } // try reading from the table - try (Table table = conn.getTable(RSGROUP_TABLE_NAME)) { - table.get(new Get(ROW_KEY)); - } - LOG.info( - "RSGroup table=" + RSGROUP_TABLE_NAME + " is online, refreshing cached information"); + FutureUtils.get(conn.getTable(RSGROUP_TABLE_NAME).get(new Get(ROW_KEY))); + LOG.info("RSGroup table={} is online, refreshing cached information", RSGROUP_TABLE_NAME); RSGroupInfoManagerImpl.this.refresh(true); online = true; // flush any inconsistencies between ZK and HTable @@ -748,8 +767,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { } else { Procedure result = masterServices.getMasterProcedureExecutor().getResult(procId); if (result != null && result.isFailed()) { - throw new IOException( - "Failed to create group table. " + MasterProcedureUtil.unwrapRemoteIOException(result)); + throw new IOException("Failed to create group table. " + + MasterProcedureUtil.unwrapRemoteIOException(result)); } } } @@ -764,33 +783,24 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager { } private void multiMutate(List mutations) throws IOException { - try (Table table = conn.getTable(RSGROUP_TABLE_NAME)) { - CoprocessorRpcChannel channel = table.coprocessorService(ROW_KEY); - MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder = - MultiRowMutationProtos.MutateRowsRequest.newBuilder(); - for (Mutation mutation : mutations) { - if (mutation instanceof Put) { - mmrBuilder.addMutationRequest(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.PUT, - mutation)); - } else if (mutation instanceof Delete) { - mmrBuilder.addMutationRequest(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.DELETE, - mutation)); - } else { - throw new DoNotRetryIOException( + MutateRowsRequest.Builder builder = MutateRowsRequest.newBuilder(); + for (Mutation mutation : mutations) { + if (mutation instanceof Put) { + builder + .addMutationRequest(ProtobufUtil.toMutation(MutationProto.MutationType.PUT, mutation)); + } else if (mutation instanceof Delete) { + builder.addMutationRequest( + ProtobufUtil.toMutation(MutationProto.MutationType.DELETE, mutation)); + } else { + throw new DoNotRetryIOException( "multiMutate doesn't support " + mutation.getClass().getName()); - } - } - - MultiRowMutationProtos.MultiRowMutationService.BlockingInterface service = - MultiRowMutationProtos.MultiRowMutationService.newBlockingStub(channel); - try { - service.mutateRows(null, mmrBuilder.build()); - } catch (ServiceException ex) { - ProtobufUtil.toIOException(ex); } } + MutateRowsRequest request = builder.build(); + AsyncTable table = conn.getTable(RSGROUP_TABLE_NAME); + FutureUtils.get(table. coprocessorService( + MultiRowMutationService::newStub, + (stub, controller, done) -> stub.mutateRows(controller, request, done), ROW_KEY)); } private void checkGroupName(String groupName) throws ConstraintException {