tables,
+ String srcGroup, String dstGroup) throws IOException;
/**
* Remove decommissioned servers from rsgroup
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
similarity index 90%
rename from hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
rename to hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
index 8aa752021e8..b54f0889822 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupInfoManagerImpl.java
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase.rsgroup;
+import com.google.protobuf.ServiceException;
import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.util.ArrayList;
@@ -33,12 +34,10 @@ 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;
@@ -46,11 +45,14 @@ 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;
@@ -60,14 +62,10 @@ 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.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.MultiRowMutationProtos;
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;
@@ -77,7 +75,6 @@ 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;
@@ -90,13 +87,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
@@ -106,24 +103,6 @@ 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 {
@@ -146,7 +125,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private volatile Map tableMap = Collections.emptyMap();
private final MasterServices masterServices;
- private final AsyncClusterConnection conn;
+ private final Connection conn;
private final ZKWatcher watcher;
private final RSGroupStartupWorker rsGroupStartupWorker;
// contains list of groups that were last flushed to persistent store
@@ -157,7 +136,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
private RSGroupInfoManagerImpl(MasterServices masterServices) throws IOException {
this.masterServices = masterServices;
this.watcher = masterServices.getZooKeeper();
- this.conn = masterServices.getAsyncClusterConnection();
+ this.conn = masterServices.getConnection();
this.rsGroupStartupWorker = new RSGroupStartupWorker();
}
@@ -370,25 +349,25 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
}
}
- private List retrieveGroupListFromGroupTable() throws IOException {
+ List retrieveGroupListFromGroupTable() throws IOException {
List rsGroupInfoList = Lists.newArrayList();
- AsyncTable> table = conn.getTable(RSGROUP_TABLE_NAME);
- try (ResultScanner scanner = table.getScanner(META_FAMILY_BYTES, META_QUALIFIER_BYTES)) {
+ try (Table table = conn.getTable(RSGROUP_TABLE_NAME);
+ ResultScanner scanner = table.getScanner(new Scan())) {
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;
}
- private List retrieveGroupListFromZookeeper() throws IOException {
- String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, RS_GROUP_ZNODE);
+ List retrieveGroupListFromZookeeper() throws IOException {
+ String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, rsGroupZNode);
List RSGroupInfoList = Lists.newArrayList();
// Overwrite any info stored by table, this takes precedence
try {
@@ -540,8 +519,7 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
resetRSGroupAndTableMaps(newGroupMap, newTableMap);
try {
- String groupBasePath =
- ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, RS_GROUP_ZNODE);
+ String groupBasePath = ZNodePaths.joinZNode(watcher.getZNodePaths().baseZNode, rsGroupZNode);
ZKUtil.createAndFailSilent(watcher, groupBasePath, ProtobufMagic.PB_MAGIC);
List zkOps = new ArrayList<>(newGroupMap.size());
@@ -724,8 +702,11 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
createRSGroupTable();
}
// try reading from the table
- FutureUtils.get(conn.getTable(RSGROUP_TABLE_NAME).get(new Get(ROW_KEY)));
- LOG.info("RSGroup table={} is online, refreshing cached information", RSGROUP_TABLE_NAME);
+ 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");
RSGroupInfoManagerImpl.this.refresh(true);
online = true;
// flush any inconsistencies between ZK and HTable
@@ -767,8 +748,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));
}
}
}
@@ -783,24 +764,33 @@ final class RSGroupInfoManagerImpl implements RSGroupInfoManager {
}
private void multiMutate(List mutations) throws IOException {
- 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(
+ 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(
"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 {
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 60887e4219c..39cf164ecf4 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(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME), newGroup);
+ groupAdmin.moveTables(Sets.newHashSet(RSGroupInfoManager.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(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME));
+ assertEquals(newGroup, groupMgr.getRSGroupOfTable(RSGroupInfoManager.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(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME).size());
+ Assert.assertEquals(0, failoverRS.getRegions(RSGroupInfoManager.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 fcaf1a79121..2ad30e48c79 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(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME);
+ .getTable(RSGroupInfoManager.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(
- RSGroupInfoManagerImpl.META_FAMILY_BYTES,
- RSGroupInfoManagerImpl.META_QUALIFIER_BYTES));
+ RSGroupInfoManager.META_FAMILY_BYTES,
+ RSGroupInfoManager.META_QUALIFIER_BYTES));
groupMap.put(proto.getName(), ProtobufUtil.toGroupInfo(proto));
}
Assert.assertEquals(Sets.newHashSet(groupMap.values()),