HBASE-22662 Move RSGroupInfoManager to hbase-server (#368)

Signed-off-by: Guanghao Zhang <zghao@apache.org>
This commit is contained in:
Duo Zhang 2019-07-11 10:34:05 +08:00
parent d36436408e
commit 3032c713e1
6 changed files with 71 additions and 83 deletions

View File

@ -106,10 +106,6 @@
<groupId>org.apache.hbase.thirdparty</groupId> <groupId>org.apache.hbase.thirdparty</groupId>
<artifactId>hbase-shaded-miscellaneous</artifactId> <artifactId>hbase-shaded-miscellaneous</artifactId>
</dependency> </dependency>
<dependency>
<groupId>com.google.protobuf</groupId>
<artifactId>protobuf-java</artifactId>
</dependency>
<dependency> <dependency>
<groupId>org.apache.zookeeper</groupId> <groupId>org.apache.zookeeper</groupId>
<artifactId>zookeeper</artifactId> <artifactId>zookeeper</artifactId>

View File

@ -121,8 +121,8 @@ public class RSGroupBasedLoadBalancer implements RSGroupableBalancer {
public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState) public List<RegionPlan> balanceCluster(Map<ServerName, List<RegionInfo>> clusterState)
throws HBaseIOException { throws HBaseIOException {
if (!isOnline()) { if (!isOnline()) {
throw new ConstraintException(RSGroupInfoManager.RSGROUP_TABLE_NAME + throw new ConstraintException(
" is not online, unable to perform balance"); RSGroupInfoManager.class.getSimpleName() + " is not online, unable to perform balance");
} }
// Calculate correct assignments and a list of RegionPlan for mis-placed regions // Calculate correct assignments and a list of RegionPlan for mis-placed regions

View File

@ -140,7 +140,7 @@ public class TestRSGroupsOfflineMode {
} }
}); });
// Move table to group and wait. // 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..."); LOG.info("Waiting for move table...");
TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() { TEST_UTIL.waitFor(WAIT_TIMEOUT, new Waiter.Predicate<Exception>() {
@Override @Override
@ -169,7 +169,7 @@ public class TestRSGroupsOfflineMode {
// Make sure balancer is in offline mode, since this is what we're testing. // Make sure balancer is in offline mode, since this is what we're testing.
assertFalse(groupMgr.isOnline()); assertFalse(groupMgr.isOnline());
// Verify the group affiliation that's loaded from ZK instead of tables. // 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)); assertEquals(RSGroupInfo.DEFAULT_GROUP, groupMgr.getRSGroupOfTable(failoverTable));
// Kill final regionserver to see the failover happens for all tables except GROUP table since // Kill final regionserver to see the failover happens for all tables except GROUP table since
// it's group does not have any online RS. // it's group does not have any online RS.
@ -182,7 +182,7 @@ public class TestRSGroupsOfflineMode {
return failoverRS.getRegions(failoverTable).size() >= 1; 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. // Need this for minicluster to shutdown cleanly.
master.stopMaster(); master.stopMaster();

View File

@ -52,7 +52,7 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
throws IOException { throws IOException {
wrapped = RSGroupAdmin; wrapped = RSGroupAdmin;
table = ConnectionFactory.createConnection(conf) table = ConnectionFactory.createConnection(conf)
.getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME); .getTable(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME);
zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null); zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null);
} }
@ -126,8 +126,8 @@ public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
RSGroupProtos.RSGroupInfo proto = RSGroupProtos.RSGroupInfo proto =
RSGroupProtos.RSGroupInfo.parseFrom( RSGroupProtos.RSGroupInfo.parseFrom(
result.getValue( result.getValue(
RSGroupInfoManager.META_FAMILY_BYTES, RSGroupInfoManagerImpl.META_FAMILY_BYTES,
RSGroupInfoManager.META_QUALIFIER_BYTES)); RSGroupInfoManagerImpl.META_QUALIFIER_BYTES));
groupMap.put(proto.getName(), ProtobufUtil.toGroupInfo(proto)); groupMap.put(proto.getName(), ProtobufUtil.toGroupInfo(proto));
} }
Assert.assertEquals(Sets.newHashSet(groupMap.values()), Assert.assertEquals(Sets.newHashSet(groupMap.values()),

View File

@ -15,38 +15,22 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.hadoop.hbase.rsgroup; package org.apache.hadoop.hbase.rsgroup;
import java.io.IOException; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.net.Address; import org.apache.hadoop.hbase.net.Address;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
/** /**
* Interface used to manage RSGroupInfo storage. An implementation * Interface used to manage RSGroupInfo storage. An implementation has the option to support offline
* has the option to support offline mode. * mode. See {@code RSGroupBasedLoadBalancer}.
* See {@link RSGroupBasedLoadBalancer}
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public interface RSGroupInfoManager { 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(); void start();
/** /**
@ -86,7 +70,6 @@ public interface RSGroupInfoManager {
/** /**
* Set the group membership of a set of tables * Set the group membership of a set of tables
*
* @param tableNames set of tables to move * @param tableNames set of tables to move
* @param groupName name of group of tables to move to * @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 * Whether the manager is able to fully return group metadata
*
* @return whether the manager is in online mode * @return whether the manager is in online mode
*/ */
boolean isOnline(); boolean isOnline();
@ -116,8 +98,8 @@ public interface RSGroupInfoManager {
* @param srcGroup groupName being moved from * @param srcGroup groupName being moved from
* @param dstGroup groupName being moved to * @param dstGroup groupName being moved to
*/ */
void moveServersAndTables(Set<Address> servers, Set<TableName> tables, void moveServersAndTables(Set<Address> servers, Set<TableName> tables, String srcGroup,
String srcGroup, String dstGroup) throws IOException; String dstGroup) throws IOException;
/** /**
* Remove decommissioned servers from rsgroup * Remove decommissioned servers from rsgroup

View File

@ -17,7 +17,6 @@
*/ */
package org.apache.hadoop.hbase.rsgroup; package org.apache.hadoop.hbase.rsgroup;
import com.google.protobuf.ServiceException;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -34,10 +33,12 @@ import java.util.SortedSet;
import java.util.TreeSet; import java.util.TreeSet;
import org.apache.hadoop.hbase.Coprocessor; import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.DoNotRetryIOException; import org.apache.hadoop.hbase.DoNotRetryIOException;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; 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.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder; import org.apache.hadoop.hbase.client.CoprocessorDescriptorBuilder;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; 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.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner; 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.TableDescriptor;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder; import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint; import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
import org.apache.hadoop.hbase.exceptions.DeserializationException; 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.MasterServices;
import org.apache.hadoop.hbase.master.ServerListener; import org.apache.hadoop.hbase.master.ServerListener;
import org.apache.hadoop.hbase.master.TableStateManager; 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.procedure2.Procedure;
import org.apache.hadoop.hbase.protobuf.ProtobufMagic; import org.apache.hadoop.hbase.protobuf.ProtobufMagic;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; 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.protobuf.generated.RSGroupProtos;
import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.DisabledRegionSplitPolicy;
import org.apache.hadoop.hbase.util.Bytes; 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.util.Threads;
import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher; import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
@ -75,6 +77,7 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; 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.Lists;
import org.apache.hbase.thirdparty.com.google.common.collect.Maps; import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
import org.apache.hbase.thirdparty.com.google.common.collect.Sets; 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 * 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 * too (in {@link #tableMap}). These Maps are persisted to the hbase:rsgroup table (and cached in
* zk) on each modification. * zk) on each modification.
* <p> * <p/>
* Mutations on state are synchronized but reads can continue without having to wait on an instance * 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 * 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). * state are read-only, just-in-case (see flushConfig).
* <p> * <p/>
* Reads must not block else there is a danger we'll deadlock. * Reads must not block else there is a danger we'll deadlock.
* <p> * <p/>
* Clients of this class, the {@link RSGroupAdminEndpoint} for example, want to query and then act * 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 * 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 * 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 { final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private static final Logger LOG = LoggerFactory.getLogger(RSGroupInfoManagerImpl.class); 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 <code>hbase:rsgroup</code> catalog table */ /** Table descriptor for <code>hbase:rsgroup</code> catalog table */
private static final TableDescriptor RSGROUP_TABLE_DESC; private static final TableDescriptor RSGROUP_TABLE_DESC;
static { static {
@ -125,7 +146,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private volatile Map<TableName, String> tableMap = Collections.emptyMap(); private volatile Map<TableName, String> tableMap = Collections.emptyMap();
private final MasterServices masterServices; private final MasterServices masterServices;
private final Connection conn; private final AsyncClusterConnection conn;
private final ZKWatcher watcher; private final ZKWatcher watcher;
private final RSGroupStartupWorker rsGroupStartupWorker; private final RSGroupStartupWorker rsGroupStartupWorker;
// contains list of groups that were last flushed to persistent store // 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 { private RSGroupInfoManagerImpl(MasterServices masterServices) throws IOException {
this.masterServices = masterServices; this.masterServices = masterServices;
this.watcher = masterServices.getZooKeeper(); this.watcher = masterServices.getZooKeeper();
this.conn = masterServices.getConnection(); this.conn = masterServices.getAsyncClusterConnection();
this.rsGroupStartupWorker = new RSGroupStartupWorker(); this.rsGroupStartupWorker = new RSGroupStartupWorker();
} }
@ -349,25 +370,25 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
} }
} }
List<RSGroupInfo> retrieveGroupListFromGroupTable() throws IOException { private List<RSGroupInfo> retrieveGroupListFromGroupTable() throws IOException {
List<RSGroupInfo> rsGroupInfoList = Lists.newArrayList(); List<RSGroupInfo> rsGroupInfoList = Lists.newArrayList();
try (Table table = conn.getTable(RSGROUP_TABLE_NAME); AsyncTable<?> table = conn.getTable(RSGROUP_TABLE_NAME);
ResultScanner scanner = table.getScanner(new Scan())) { try (ResultScanner scanner = table.getScanner(META_FAMILY_BYTES, META_QUALIFIER_BYTES)) {
for (Result result;;) { for (Result result;;) {
result = scanner.next(); result = scanner.next();
if (result == null) { if (result == null) {
break; break;
} }
RSGroupProtos.RSGroupInfo proto = RSGroupProtos.RSGroupInfo 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)); rsGroupInfoList.add(ProtobufUtil.toGroupInfo(proto));
} }
} }
return rsGroupInfoList; return rsGroupInfoList;
} }
List<RSGroupInfo> retrieveGroupListFromZookeeper() throws IOException { private List<RSGroupInfo> retrieveGroupListFromZookeeper() throws IOException {
String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, rsGroupZNode); String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, RS_GROUP_ZNODE);
List<RSGroupInfo> RSGroupInfoList = Lists.newArrayList(); List<RSGroupInfo> RSGroupInfoList = Lists.newArrayList();
// Overwrite any info stored by table, this takes precedence // Overwrite any info stored by table, this takes precedence
try { try {
@ -519,7 +540,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
resetRSGroupAndTableMaps(newGroupMap, newTableMap); resetRSGroupAndTableMaps(newGroupMap, newTableMap);
try { 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); ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC);
List<ZKUtil.ZKUtilOp> zkOps = new ArrayList<>(newGroupMap.size()); List<ZKUtil.ZKUtilOp> zkOps = new ArrayList<>(newGroupMap.size());
@ -702,11 +724,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
createRSGroupTable(); createRSGroupTable();
} }
// try reading from the table // try reading from the table
try (Table table = conn.getTable(RSGROUP_TABLE_NAME)) { FutureUtils.get(conn.getTable(RSGROUP_TABLE_NAME).get(new Get(ROW_KEY)));
table.get(new Get(ROW_KEY)); LOG.info("RSGroup table={} is online, refreshing cached information", RSGROUP_TABLE_NAME);
}
LOG.info(
"RSGroup table=" + RSGROUP_TABLE_NAME + " is online, refreshing cached information");
RSGroupInfoManagerImpl.this.refresh(true); RSGroupInfoManagerImpl.this.refresh(true);
online = true; online = true;
// flush any inconsistencies between ZK and HTable // flush any inconsistencies between ZK and HTable
@ -748,8 +767,8 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
} else { } else {
Procedure<?> result = masterServices.getMasterProcedureExecutor().getResult(procId); Procedure<?> result = masterServices.getMasterProcedureExecutor().getResult(procId);
if (result != null && result.isFailed()) { if (result != null && result.isFailed()) {
throw new IOException( throw new IOException("Failed to create group table. " +
"Failed to create group table. " + MasterProcedureUtil.unwrapRemoteIOException(result)); MasterProcedureUtil.unwrapRemoteIOException(result));
} }
} }
} }
@ -764,33 +783,24 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
} }
private void multiMutate(List<Mutation> mutations) throws IOException { private void multiMutate(List<Mutation> mutations) throws IOException {
try (Table table = conn.getTable(RSGROUP_TABLE_NAME)) { MutateRowsRequest.Builder builder = MutateRowsRequest.newBuilder();
CoprocessorRpcChannel channel = table.coprocessorService(ROW_KEY); for (Mutation mutation : mutations) {
MultiRowMutationProtos.MutateRowsRequest.Builder mmrBuilder = if (mutation instanceof Put) {
MultiRowMutationProtos.MutateRowsRequest.newBuilder(); builder
for (Mutation mutation : mutations) { .addMutationRequest(ProtobufUtil.toMutation(MutationProto.MutationType.PUT, mutation));
if (mutation instanceof Put) { } else if (mutation instanceof Delete) {
mmrBuilder.addMutationRequest(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation( builder.addMutationRequest(
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.PUT, ProtobufUtil.toMutation(MutationProto.MutationType.DELETE, mutation));
mutation)); } else {
} else if (mutation instanceof Delete) { throw new DoNotRetryIOException(
mmrBuilder.addMutationRequest(org.apache.hadoop.hbase.protobuf.ProtobufUtil.toMutation(
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType.DELETE,
mutation));
} else {
throw new DoNotRetryIOException(
"multiMutate doesn't support " + mutation.getClass().getName()); "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.<MultiRowMutationService, MutateRowsResponse> coprocessorService(
MultiRowMutationService::newStub,
(stub, controller, done) -> stub.mutateRows(controller, request, done), ROW_KEY));
} }
private void checkGroupName(String groupName) throws ConstraintException { private void checkGroupName(String groupName) throws ConstraintException {