HBASE-15295 MutateTableAccess.multiMutate() does not get high priority causing a deadlock
Conflicts: hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/CoprocessorRpcChannel.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/MasterCoprocessorRpcChannel.java hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RegionCoprocessorRpcChannel.java hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/ProtobufUtil.java hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/HConnectionTestingUtility.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFromClientSide3.java hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestScannersFromClientSide.java hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestClockSkewDetection.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestNamespaceCommands.java
This commit is contained in:
parent
5851ad0230
commit
5202d3c25b
|
@ -229,6 +229,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
|
|||
private TableName tableName = null;
|
||||
|
||||
/** HRegionInfo for first meta region */
|
||||
// TODO: How come Meta regions still do not have encoded region names? Fix.
|
||||
public static final HRegionInfo FIRST_META_REGIONINFO =
|
||||
new HRegionInfo(1L, TableName.META_TABLE_NAME);
|
||||
|
||||
|
|
|
@ -87,7 +87,7 @@ public class BufferedMutatorImpl implements BufferedMutator {
|
|||
this.pool = params.getPool();
|
||||
this.listener = params.getListener();
|
||||
|
||||
TableConfiguration tableConf = new TableConfiguration(conf);
|
||||
ConnectionConfiguration tableConf = new ConnectionConfiguration(conf);
|
||||
this.writeBufferSize = params.getWriteBufferSize() != BufferedMutatorParams.UNSET ?
|
||||
params.getWriteBufferSize() : tableConf.getWriteBufferSize();
|
||||
this.maxKeyValueSize = params.getMaxKeyValueSize() != BufferedMutatorParams.UNSET ?
|
||||
|
|
|
@ -30,11 +30,12 @@ import org.apache.hadoop.hbase.TableName;
|
|||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
||||
|
||||
/** Internal methods on HConnection that should not be used by user code. */
|
||||
/** Internal methods on Connection that should not be used by user code. */
|
||||
@InterfaceAudience.Private
|
||||
// NOTE: Although this class is public, this class is meant to be used directly from internal
|
||||
// classes and unit tests only.
|
||||
|
@ -284,6 +285,21 @@ public interface ClusterConnection extends HConnection {
|
|||
*/
|
||||
RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
|
||||
|
||||
/**
|
||||
* @return Connection's RpcRetryingCallerFactory instance
|
||||
*/
|
||||
RpcRetryingCallerFactory getRpcRetryingCallerFactory();
|
||||
|
||||
/**
|
||||
* @return Connection's RpcControllerFactory instance
|
||||
*/
|
||||
RpcControllerFactory getRpcControllerFactory();
|
||||
|
||||
/**
|
||||
* @return a ConnectionConfiguration object holding parsed configuration values
|
||||
*/
|
||||
ConnectionConfiguration getConnectionConfiguration();
|
||||
|
||||
/**
|
||||
*
|
||||
* @return true if this is a managed connection.
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch.Callback;
|
|||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
|
||||
/**
|
||||
* An internal class that delegates to an {@link HConnection} instance.
|
||||
|
@ -474,4 +475,19 @@ abstract class ConnectionAdapter implements ClusterConnection {
|
|||
public boolean hasCellBlockSupport() {
|
||||
return wrappedConnection.hasCellBlockSupport();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConnectionConfiguration getConnectionConfiguration() {
|
||||
return wrappedConnection.getConnectionConfiguration();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
|
||||
return wrappedConnection.getRpcRetryingCallerFactory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public RpcControllerFactory getRpcControllerFactory() {
|
||||
return wrappedConnection.getRpcControllerFactory();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,15 +18,15 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
/**
|
||||
*
|
||||
* Configuration parameters for the connection.
|
||||
* Configuration is a heavy weight registry that does a lot of string operations and regex matching.
|
||||
* Method calls into Configuration account for high CPU usage and have huge performance impact.
|
||||
* This class caches the value in the TableConfiguration object to improve performance.
|
||||
* This class caches connection-related configuration values in the ConnectionConfiguration
|
||||
* object so that expensive conf.getXXX() calls are avoided every time HTable, etc is instantiated.
|
||||
* see HBASE-12128
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class TableConfiguration {
|
||||
public class ConnectionConfiguration {
|
||||
|
||||
public static final String WRITE_BUFFER_SIZE_KEY = "hbase.client.write.buffer";
|
||||
public static final long WRITE_BUFFER_SIZE_DEFAULT = 2097152;
|
||||
|
@ -47,7 +47,7 @@ public class TableConfiguration {
|
|||
* Constructor
|
||||
* @param conf Configuration object
|
||||
*/
|
||||
TableConfiguration(Configuration conf) {
|
||||
ConnectionConfiguration(Configuration conf) {
|
||||
this.writeBufferSize = conf.getLong(WRITE_BUFFER_SIZE_KEY, WRITE_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
this.metaOperationTimeout = conf.getInt(
|
||||
|
@ -82,7 +82,7 @@ public class TableConfiguration {
|
|||
* In real usage, we should read the configuration from the Configuration object.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected TableConfiguration() {
|
||||
protected ConnectionConfiguration() {
|
||||
this.writeBufferSize = WRITE_BUFFER_SIZE_DEFAULT;
|
||||
this.metaOperationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
|
||||
this.operationTimeout = HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT;
|
|
@ -595,7 +595,7 @@ class ConnectionManager {
|
|||
|
||||
// cache the configuration value for tables so that we can avoid calling
|
||||
// the expensive Configuration to fetch the value multiple times.
|
||||
private final TableConfiguration tableConfig;
|
||||
private final ConnectionConfiguration connectionConfig;
|
||||
|
||||
// Client rpc instance.
|
||||
private RpcClient rpcClient;
|
||||
|
@ -644,13 +644,13 @@ class ConnectionManager {
|
|||
this.user = user;
|
||||
this.batchPool = pool;
|
||||
this.managed = managed;
|
||||
this.tableConfig = new TableConfiguration(conf);
|
||||
this.connectionConfig = new ConnectionConfiguration(conf);
|
||||
this.closed = false;
|
||||
this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
this.useMetaReplicas = conf.getBoolean(HConstants.USE_META_REPLICAS,
|
||||
HConstants.DEFAULT_USE_META_REPLICAS);
|
||||
this.numTries = tableConfig.getRetriesNumber();
|
||||
this.numTries = connectionConfig.getRetriesNumber();
|
||||
this.rpcTimeout = conf.getInt(
|
||||
HConstants.HBASE_RPC_TIMEOUT_KEY,
|
||||
HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
|
||||
|
@ -746,7 +746,7 @@ class ConnectionManager {
|
|||
if (managed) {
|
||||
throw new NeedUnmanagedConnectionException();
|
||||
}
|
||||
return new HTable(tableName, this, tableConfig, rpcCallerFactory, rpcControllerFactory, pool);
|
||||
return new HTable(tableName, this, connectionConfig, rpcCallerFactory, rpcControllerFactory, pool);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -758,10 +758,10 @@ class ConnectionManager {
|
|||
params.pool(HTable.getDefaultExecutor(getConfiguration()));
|
||||
}
|
||||
if (params.getWriteBufferSize() == BufferedMutatorParams.UNSET) {
|
||||
params.writeBufferSize(tableConfig.getWriteBufferSize());
|
||||
params.writeBufferSize(connectionConfig.getWriteBufferSize());
|
||||
}
|
||||
if (params.getMaxKeyValueSize() == BufferedMutatorParams.UNSET) {
|
||||
params.maxKeyValueSize(tableConfig.getMaxKeyValueSize());
|
||||
params.maxKeyValueSize(connectionConfig.getMaxKeyValueSize());
|
||||
}
|
||||
return new BufferedMutatorImpl(this, rpcCallerFactory, rpcControllerFactory, params);
|
||||
}
|
||||
|
@ -2642,6 +2642,21 @@ class ConnectionManager {
|
|||
public boolean hasCellBlockSupport() {
|
||||
return this.rpcClient.hasCellBlockSupport();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ConnectionConfiguration getConnectionConfiguration() {
|
||||
return this.connectionConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RpcRetryingCallerFactory getRpcRetryingCallerFactory() {
|
||||
return this.rpcCallerFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RpcControllerFactory getRpcControllerFactory() {
|
||||
return this.rpcControllerFactory;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -72,7 +72,9 @@ import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
|||
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
|
||||
|
@ -214,6 +216,7 @@ public class HBaseAdmin implements Admin {
|
|||
private int operationTimeout;
|
||||
|
||||
private RpcRetryingCallerFactory rpcCallerFactory;
|
||||
private RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
private NonceGenerator ng;
|
||||
|
||||
|
@ -261,6 +264,7 @@ public class HBaseAdmin implements Admin {
|
|||
this.conf = connection.getConfiguration();
|
||||
this.connection = connection;
|
||||
|
||||
// TODO: receive ConnectionConfiguration here rather than re-parsing these configs every time.
|
||||
this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
|
||||
this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
|
||||
|
@ -272,7 +276,8 @@ public class HBaseAdmin implements Admin {
|
|||
this.syncWaitTimeout = this.conf.getInt(
|
||||
"hbase.client.sync.wait.timeout.msec", 10 * 60000); // 10min
|
||||
|
||||
this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
|
||||
this.rpcCallerFactory = connection.getRpcRetryingCallerFactory();
|
||||
this.rpcControllerFactory = connection.getRpcControllerFactory();
|
||||
|
||||
this.ng = this.connection.getNonceGenerator();
|
||||
}
|
||||
|
@ -336,9 +341,11 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<AbortProcedureResponse>(getConnection()) {
|
||||
@Override
|
||||
public AbortProcedureResponse call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
AbortProcedureRequest abortProcRequest =
|
||||
AbortProcedureRequest.newBuilder().setProcId(procId).build();
|
||||
return master.abortProcedure(null,abortProcRequest);
|
||||
return master.abortProcedure(controller, abortProcRequest);
|
||||
}
|
||||
}).getIsProcedureAborted();
|
||||
|
||||
|
@ -428,9 +435,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
|
||||
@Override
|
||||
public HTableDescriptor[] call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
|
||||
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
|
||||
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -507,9 +516,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
|
||||
@Override
|
||||
public TableName[] call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
GetTableNamesRequest req =
|
||||
RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
|
||||
return ProtobufUtil.getTableNameArray(master.getTableNames(null, req)
|
||||
return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
|
||||
.getTableNamesList());
|
||||
}
|
||||
});
|
||||
|
@ -531,21 +542,24 @@ public class HBaseAdmin implements Admin {
|
|||
@Override
|
||||
public HTableDescriptor getTableDescriptor(final TableName tableName)
|
||||
throws TableNotFoundException, IOException {
|
||||
return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, operationTimeout);
|
||||
return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
|
||||
operationTimeout);
|
||||
}
|
||||
|
||||
static HTableDescriptor getTableDescriptor(final TableName tableName,
|
||||
HConnection connection, RpcRetryingCallerFactory rpcCallerFactory,
|
||||
static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
|
||||
RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
|
||||
int operationTimeout) throws TableNotFoundException, IOException {
|
||||
|
||||
if (tableName == null) return null;
|
||||
HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
|
||||
@Override
|
||||
public HTableDescriptor call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
GetTableDescriptorsResponse htds;
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(tableName);
|
||||
htds = master.getTableDescriptors(null, req);
|
||||
htds = master.getTableDescriptors(controller, req);
|
||||
|
||||
if (!htds.getTableSchemaList().isEmpty()) {
|
||||
return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
|
||||
|
@ -738,9 +752,12 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<CreateTableResponse>(getConnection()) {
|
||||
@Override
|
||||
public CreateTableResponse call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(desc.getTableName());
|
||||
CreateTableRequest request = RequestConverter.buildCreateTableRequest(
|
||||
desc, splitKeys, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.createTable(null, request);
|
||||
return master.createTable(controller, request);
|
||||
}
|
||||
});
|
||||
return new CreateTableFuture(this, desc, splitKeys, response);
|
||||
|
@ -909,9 +926,12 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<DeleteTableResponse>(getConnection()) {
|
||||
@Override
|
||||
public DeleteTableResponse call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
DeleteTableRequest req =
|
||||
RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.deleteTable(null,req);
|
||||
RequestConverter.buildDeleteTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
|
||||
return master.deleteTable(controller,req);
|
||||
}
|
||||
});
|
||||
return new DeleteTableFuture(this, tableName, response);
|
||||
|
@ -1157,10 +1177,14 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<EnableTableResponse>(getConnection()) {
|
||||
@Override
|
||||
public EnableTableResponse call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
|
||||
LOG.info("Started enable of " + tableName);
|
||||
EnableTableRequest req =
|
||||
RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.enableTable(null,req);
|
||||
RequestConverter.buildEnableTableRequest(tableName, ng.getNonceGroup(),ng.newNonce());
|
||||
return master.enableTable(controller,req);
|
||||
}
|
||||
});
|
||||
return new EnableTableFuture(this, tableName, response);
|
||||
|
@ -1345,10 +1369,15 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<DisableTableResponse>(getConnection()) {
|
||||
@Override
|
||||
public DisableTableResponse call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
|
||||
LOG.info("Started disable of " + tableName);
|
||||
DisableTableRequest req =
|
||||
RequestConverter.buildDisableTableRequest(tableName, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.disableTable(null, req);
|
||||
RequestConverter.buildDisableTableRequest(
|
||||
tableName, ng.getNonceGroup(), ng.newNonce());
|
||||
return master.disableTable(controller, req);
|
||||
}
|
||||
});
|
||||
return new DisableTableFuture(this, tableName, response);
|
||||
|
@ -1557,9 +1586,13 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
|
||||
@Override
|
||||
public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
|
||||
GetSchemaAlterStatusRequest req = RequestConverter
|
||||
.buildGetSchemaAlterStatusRequest(tableName);
|
||||
GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
|
||||
GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
|
||||
Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
|
||||
.getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
|
||||
return pair;
|
||||
|
@ -1598,7 +1631,6 @@ public class HBaseAdmin implements Admin {
|
|||
addColumn(TableName.valueOf(tableName), column);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Add a column to an existing table.
|
||||
* Asynchronous operation.
|
||||
|
@ -1626,9 +1658,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
AddColumnRequest req = RequestConverter.buildAddColumnRequest(
|
||||
tableName, column, ng.getNonceGroup(), ng.newNonce());
|
||||
master.addColumn(null,req);
|
||||
master.addColumn(controller,req);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -1674,9 +1709,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(
|
||||
tableName, columnName, ng.getNonceGroup(), ng.newNonce());
|
||||
master.deleteColumn(null,req);
|
||||
master.deleteColumn(controller, req);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -1708,8 +1746,6 @@ public class HBaseAdmin implements Admin {
|
|||
modifyColumn(TableName.valueOf(tableName), descriptor);
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Modify an existing column family on a table.
|
||||
* Asynchronous operation.
|
||||
|
@ -1724,9 +1760,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(
|
||||
tableName, descriptor, ng.getNonceGroup(), ng.newNonce());
|
||||
master.modifyColumn(null,req);
|
||||
master.modifyColumn(controller, req);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -1812,7 +1851,10 @@ public class HBaseAdmin implements Admin {
|
|||
CloseRegionRequest request =
|
||||
RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
|
||||
try {
|
||||
CloseRegionResponse response = admin.closeRegion(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
CloseRegionResponse response = admin.closeRegion(controller, request);
|
||||
boolean isRegionClosed = response.getClosed();
|
||||
if (false == isRegionClosed) {
|
||||
LOG.error("Not able to close the region " + encodedRegionName + ".");
|
||||
|
@ -1834,8 +1876,10 @@ public class HBaseAdmin implements Admin {
|
|||
public void closeRegion(final ServerName sn, final HRegionInfo hri)
|
||||
throws IOException {
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
|
||||
// Close the region without updating zk state.
|
||||
ProtobufUtil.closeRegion(admin, sn, hri.getRegionName(), false);
|
||||
ProtobufUtil.closeRegion(controller, admin, sn, hri.getRegionName(), false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1844,7 +1888,8 @@ public class HBaseAdmin implements Admin {
|
|||
@Override
|
||||
public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
return ProtobufUtil.getOnlineRegions(admin);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
return ProtobufUtil.getOnlineRegions(controller, admin);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1903,11 +1948,12 @@ public class HBaseAdmin implements Admin {
|
|||
|
||||
private void flush(final ServerName sn, final HRegionInfo hri)
|
||||
throws IOException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
FlushRegionRequest request =
|
||||
RequestConverter.buildFlushRegionRequest(hri.getRegionName());
|
||||
try {
|
||||
admin.flushRegion(null, request);
|
||||
admin.flushRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -2159,11 +2205,13 @@ public class HBaseAdmin implements Admin {
|
|||
private void compact(final ServerName sn, final HRegionInfo hri,
|
||||
final boolean major, final byte [] family)
|
||||
throws IOException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
CompactRegionRequest request =
|
||||
RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
|
||||
try {
|
||||
admin.compactRegion(null, request);
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
admin.compactRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -2189,10 +2237,17 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
// Hard to know the table name, at least check if meta
|
||||
if (isMetaRegion(encodedRegionName)) {
|
||||
controller.setPriority(TableName.META_TABLE_NAME);
|
||||
}
|
||||
|
||||
try {
|
||||
MoveRegionRequest request =
|
||||
RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
|
||||
master.moveRegion(null, request);
|
||||
master.moveRegion(controller, request);
|
||||
} catch (DeserializationException de) {
|
||||
LOG.error("Could not parse destination server name: " + de);
|
||||
throw new ServiceException(new DoNotRetryIOException(de));
|
||||
|
@ -2202,6 +2257,11 @@ public class HBaseAdmin implements Admin {
|
|||
});
|
||||
}
|
||||
|
||||
private boolean isMetaRegion(final byte[] regionName) {
|
||||
return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
|
||||
|| Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param regionName
|
||||
* Region name to assign.
|
||||
|
@ -2216,9 +2276,16 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
// Hard to know the table name, at least check if meta
|
||||
if (isMetaRegion(regionName)) {
|
||||
controller.setPriority(TableName.META_TABLE_NAME);
|
||||
}
|
||||
|
||||
AssignRegionRequest request =
|
||||
RequestConverter.buildAssignRegionRequest(toBeAssigned);
|
||||
master.assignRegion(null,request);
|
||||
master.assignRegion(controller,request);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2245,9 +2312,15 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
// Hard to know the table name, at least check if meta
|
||||
if (isMetaRegion(regionName)) {
|
||||
controller.setPriority(TableName.META_TABLE_NAME);
|
||||
}
|
||||
UnassignRegionRequest request =
|
||||
RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
|
||||
master.unassignRegion(null, request);
|
||||
master.unassignRegion(controller, request);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2271,7 +2344,13 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
// Hard to know the table name, at least check if meta
|
||||
if (isMetaRegion(regionName)) {
|
||||
controller.setPriority(TableName.META_TABLE_NAME);
|
||||
}
|
||||
master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2289,9 +2368,12 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
SetBalancerRunningRequest req =
|
||||
RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
|
||||
return master.setBalancerRunning(null, req).getPrevBalanceValue();
|
||||
return master.setBalancerRunning(controller, req).getPrevBalanceValue();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2307,7 +2389,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.balance(null, RequestConverter.buildBalanceRequest(false)).getBalancerRan();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.balance(controller,
|
||||
RequestConverter.buildBalanceRequest(false)).getBalancerRan();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2317,7 +2403,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.balance(null, RequestConverter.buildBalanceRequest(force)).getBalancerRan();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.balance(controller,
|
||||
RequestConverter.buildBalanceRequest(force)).getBalancerRan();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2333,8 +2423,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.isBalancerEnabled(null, RequestConverter.buildIsBalancerEnabledRequest())
|
||||
.getEnabled();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.isBalancerEnabled(controller,
|
||||
RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2349,7 +2442,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.normalize(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.normalize(controller,
|
||||
RequestConverter.buildNormalizeRequest()).getNormalizerRan();
|
||||
}
|
||||
});
|
||||
|
@ -2364,7 +2460,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.isNormalizerEnabled(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.isNormalizerEnabled(controller,
|
||||
RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
|
||||
}
|
||||
});
|
||||
|
@ -2379,9 +2478,12 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
SetNormalizerRunningRequest req =
|
||||
RequestConverter.buildSetNormalizerRunningRequest(on);
|
||||
return master.setNormalizerRunning(null, req).getPrevNormalizerValue();
|
||||
return master.setNormalizerRunning(controller, req).getPrevNormalizerValue();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2398,7 +2500,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.enableCatalogJanitor(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.enableCatalogJanitor(controller,
|
||||
RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
|
||||
}
|
||||
});
|
||||
|
@ -2414,7 +2519,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Integer>(getConnection()) {
|
||||
@Override
|
||||
public Integer call(int callTimeout) throws ServiceException {
|
||||
return master.runCatalogScan(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.runCatalogScan(controller,
|
||||
RequestConverter.buildCatalogScanRequest()).getScanResult();
|
||||
}
|
||||
});
|
||||
|
@ -2429,7 +2537,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Boolean>(getConnection()) {
|
||||
@Override
|
||||
public Boolean call(int callTimeout) throws ServiceException {
|
||||
return master.isCatalogJanitorEnabled(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
return master.isCatalogJanitorEnabled(controller,
|
||||
RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
|
||||
}
|
||||
});
|
||||
|
@ -2474,11 +2585,14 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
|
||||
try {
|
||||
DispatchMergingRegionsRequest request = RequestConverter
|
||||
.buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
|
||||
encodedNameOfRegionB, forcible);
|
||||
master.dispatchMergingRegions(null, request);
|
||||
master.dispatchMergingRegions(controller, request);
|
||||
} catch (DeserializationException de) {
|
||||
LOG.error("Could not parse destination server name: " + de);
|
||||
}
|
||||
|
@ -2610,9 +2724,12 @@ public class HBaseAdmin implements Admin {
|
|||
Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
|
||||
throw new IOException("should not give a splitkey which equals to startkey!");
|
||||
}
|
||||
// TODO: This is not executed via retries
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setPriority(hri.getTable());
|
||||
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
ProtobufUtil.split(admin, hri, splitPoint);
|
||||
ProtobufUtil.split(controller, admin, hri, splitPoint);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2635,9 +2752,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(tableName);
|
||||
ModifyTableRequest request = RequestConverter.buildModifyTableRequest(
|
||||
tableName, htd, ng.getNonceGroup(), ng.newNonce());
|
||||
master.modifyTable(null, request);
|
||||
master.modifyTable(controller, request);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2750,7 +2870,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
master.shutdown(null,ShutdownRequest.newBuilder().build());
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(HConstants.HIGH_QOS);
|
||||
master.shutdown(controller, ShutdownRequest.newBuilder().build());
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2767,7 +2890,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
master.stopMaster(null, StopMasterRequest.newBuilder().build());
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
controller.setPriority(HConstants.HIGH_QOS);
|
||||
master.stopMaster(controller, StopMasterRequest.newBuilder().build());
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -2788,8 +2914,12 @@ public class HBaseAdmin implements Admin {
|
|||
this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
|
||||
StopServerRequest request = RequestConverter.buildStopServerRequest(
|
||||
"Called by admin client " + this.connection.toString());
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
|
||||
controller.setPriority(HConstants.HIGH_QOS);
|
||||
try {
|
||||
admin.stopServer(null, request);
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
admin.stopServer(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -2805,8 +2935,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
|
||||
@Override
|
||||
public ClusterStatus call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
|
||||
return ClusterStatus.convert(master.getClusterStatus(null, req).getClusterStatus());
|
||||
return ClusterStatus.convert(master.getClusterStatus(controller, req).getClusterStatus());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -2829,7 +2961,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws Exception {
|
||||
master.createNamespace(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
// TODO: set priority based on NS?
|
||||
master.createNamespace(controller,
|
||||
CreateNamespaceRequest.newBuilder()
|
||||
.setNamespaceDescriptor(ProtobufUtil
|
||||
.toProtoNamespaceDescriptor(descriptor)).build()
|
||||
|
@ -2849,7 +2984,9 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws Exception {
|
||||
master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
|
||||
setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
|
||||
return null;
|
||||
}
|
||||
|
@ -2866,7 +3003,9 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws Exception {
|
||||
master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
|
||||
setNamespaceName(name).build());
|
||||
return null;
|
||||
}
|
||||
|
@ -2885,8 +3024,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
|
||||
@Override
|
||||
public NamespaceDescriptor call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return ProtobufUtil.toNamespaceDescriptor(
|
||||
master.getNamespaceDescriptor(null, GetNamespaceDescriptorRequest.newBuilder().
|
||||
master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
|
||||
setNamespaceName(name).build()).getNamespaceDescriptor());
|
||||
}
|
||||
});
|
||||
|
@ -2903,9 +3044,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
|
||||
@Override
|
||||
public NamespaceDescriptor[] call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
List<HBaseProtos.NamespaceDescriptor> list =
|
||||
master.listNamespaceDescriptors(null, ListNamespaceDescriptorsRequest.newBuilder().
|
||||
build()).getNamespaceDescriptorList();
|
||||
master.listNamespaceDescriptors(controller,
|
||||
ListNamespaceDescriptorsRequest.newBuilder().build())
|
||||
.getNamespaceDescriptorList();
|
||||
NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
|
||||
for(int i = 0; i < list.size(); i++) {
|
||||
res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
|
||||
|
@ -2926,8 +3070,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
|
||||
@Override
|
||||
public ProcedureInfo[] call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
List<ProcedureProtos.Procedure> procList = master.listProcedures(
|
||||
null, ListProceduresRequest.newBuilder().build()).getProcedureList();
|
||||
controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
|
||||
ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
|
||||
for (int i = 0; i < procList.size(); i++) {
|
||||
procInfoList[i] = ProcedureInfo.convert(procList.get(i));
|
||||
|
@ -2949,9 +3095,12 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
|
||||
@Override
|
||||
public HTableDescriptor[] call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
List<TableSchema> list =
|
||||
master.listTableDescriptorsByNamespace(null, ListTableDescriptorsByNamespaceRequest.
|
||||
newBuilder().setNamespaceName(name).build()).getTableSchemaList();
|
||||
master.listTableDescriptorsByNamespace(controller,
|
||||
ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
|
||||
.build()).getTableSchemaList();
|
||||
HTableDescriptor[] res = new HTableDescriptor[list.size()];
|
||||
for(int i=0; i < list.size(); i++) {
|
||||
|
||||
|
@ -2974,8 +3123,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<TableName[]>(getConnection()) {
|
||||
@Override
|
||||
public TableName[] call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
List<HBaseProtos.TableName> tableNames =
|
||||
master.listTableNamesByNamespace(null, ListTableNamesByNamespaceRequest.
|
||||
master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
|
||||
newBuilder().setNamespaceName(name).build())
|
||||
.getTableNameList();
|
||||
TableName[] result = new TableName[tableNames.size()];
|
||||
|
@ -3073,9 +3224,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
|
||||
@Override
|
||||
public HTableDescriptor[] call(int callTimeout) throws Exception {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
GetTableDescriptorsRequest req =
|
||||
RequestConverter.buildGetTableDescriptorsRequest(tableNames);
|
||||
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(null, req));
|
||||
return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -3121,8 +3274,11 @@ public class HBaseAdmin implements Admin {
|
|||
FailedLogCloseException {
|
||||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
|
||||
try {
|
||||
return admin.rollWALWriter(null, request);
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
return admin.rollWALWriter(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -3267,7 +3423,9 @@ public class HBaseAdmin implements Admin {
|
|||
AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
|
||||
GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
|
||||
regionServerPair.getFirst().getRegionName(), true);
|
||||
GetRegionInfoResponse response = admin.getRegionInfo(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
// TODO: this does not do retries, it should. Set priority and timeout in controller
|
||||
GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
|
||||
return response.getCompactionState();
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
|
@ -3475,7 +3633,9 @@ public class HBaseAdmin implements Admin {
|
|||
done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
|
||||
@Override
|
||||
public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
|
||||
return master.isSnapshotDone(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.isSnapshotDone(controller, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -3505,7 +3665,9 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
|
||||
@Override
|
||||
public SnapshotResponse call(int callTimeout) throws ServiceException {
|
||||
return master.snapshot(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.snapshot(controller, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -3537,7 +3699,9 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
|
||||
@Override
|
||||
public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
|
||||
return master.isSnapshotDone(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.isSnapshotDone(controller,
|
||||
IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
|
||||
}
|
||||
}).getDone();
|
||||
|
@ -3792,7 +3956,9 @@ public class HBaseAdmin implements Admin {
|
|||
getConnection()) {
|
||||
@Override
|
||||
public ExecProcedureResponse call(int callTimeout) throws ServiceException {
|
||||
return master.execProcedureWithRet(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.execProcedureWithRet(controller, request);
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -3826,7 +3992,9 @@ public class HBaseAdmin implements Admin {
|
|||
getConnection()) {
|
||||
@Override
|
||||
public ExecProcedureResponse call(int callTimeout) throws ServiceException {
|
||||
return master.execProcedure(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.execProcedure(controller, request);
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -3890,7 +4058,9 @@ public class HBaseAdmin implements Admin {
|
|||
new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
|
||||
@Override
|
||||
public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
|
||||
return master.isProcedureDone(null, IsProcedureDoneRequest
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.isProcedureDone(controller, IsProcedureDoneRequest
|
||||
.newBuilder().setProcedure(desc).build());
|
||||
}
|
||||
}).getDone();
|
||||
|
@ -3936,7 +4106,9 @@ public class HBaseAdmin implements Admin {
|
|||
getConnection()) {
|
||||
@Override
|
||||
public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
|
||||
return master.isRestoreSnapshotDone(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.isRestoreSnapshotDone(controller, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -3966,7 +4138,9 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
|
||||
@Override
|
||||
public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
|
||||
return master.restoreSnapshot(null, request);
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.restoreSnapshot(controller, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -3981,8 +4155,10 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
|
||||
@Override
|
||||
public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
|
||||
return master.getCompletedSnapshots(null, GetCompletedSnapshotsRequest.newBuilder().build())
|
||||
.getSnapshotsList();
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.getCompletedSnapshots(controller,
|
||||
GetCompletedSnapshotsRequest.newBuilder().build()).getSnapshotsList();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -4080,7 +4256,9 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
master.deleteSnapshot(null,
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
master.deleteSnapshot(controller,
|
||||
DeleteSnapshotRequest.newBuilder().
|
||||
setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build()
|
||||
);
|
||||
|
@ -4122,8 +4300,10 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
this.master.deleteSnapshot(null, DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot)
|
||||
.build());
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
|
||||
.setSnapshot(snapshot).build());
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -4173,7 +4353,9 @@ public class HBaseAdmin implements Admin {
|
|||
executeCallable(new MasterCallable<Void>(getConnection()) {
|
||||
@Override
|
||||
public Void call(int callTimeout) throws ServiceException {
|
||||
this.master.setQuota(null, QuotaSettings.buildSetQuotaRequestProto(quota));
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -4308,10 +4490,12 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Long>(getConnection()) {
|
||||
@Override
|
||||
public Long call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
MajorCompactionTimestampRequest req =
|
||||
MajorCompactionTimestampRequest.newBuilder()
|
||||
.setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
|
||||
return master.getLastMajorCompactionTimestamp(null, req).getCompactionTimestamp();
|
||||
return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -4321,13 +4505,16 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<Long>(getConnection()) {
|
||||
@Override
|
||||
public Long call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
MajorCompactionTimestampForRegionRequest req =
|
||||
MajorCompactionTimestampForRegionRequest
|
||||
.newBuilder()
|
||||
.setRegion(
|
||||
RequestConverter
|
||||
.buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
|
||||
return master.getLastMajorCompactionTimestampForRegion(null, req).getCompactionTimestamp();
|
||||
return master.getLastMajorCompactionTimestampForRegion(controller, req)
|
||||
.getCompactionTimestamp();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -4386,7 +4573,9 @@ public class HBaseAdmin implements Admin {
|
|||
admin.getConnection()) {
|
||||
@Override
|
||||
public AbortProcedureResponse call(int callTimeout) throws ServiceException {
|
||||
return master.abortProcedure(null, request);
|
||||
PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
return master.abortProcedure(controller, request);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
@ -4598,9 +4787,11 @@ public class HBaseAdmin implements Admin {
|
|||
return executeCallable(new MasterCallable<List<SecurityCapability>>(getConnection()) {
|
||||
@Override
|
||||
public List<SecurityCapability> call(int callTimeout) throws ServiceException {
|
||||
PayloadCarryingRpcController controller = rpcControllerFactory.newController();
|
||||
controller.setCallTimeout(callTimeout);
|
||||
SecurityCapabilitiesRequest req = SecurityCapabilitiesRequest.newBuilder().build();
|
||||
return ProtobufUtil.toSecurityCapabilityList(
|
||||
master.getSecurityCapabilities(null, req).getCapabilitiesList());
|
||||
master.getSecurityCapabilities(controller, req).getCapabilitiesList());
|
||||
}
|
||||
});
|
||||
} catch (IOException e) {
|
||||
|
@ -4641,4 +4832,7 @@ public class HBaseAdmin implements Admin {
|
|||
});
|
||||
}
|
||||
|
||||
private RpcControllerFactory getRpcControllerFactory() {
|
||||
return rpcControllerFactory;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -117,7 +117,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
protected ClusterConnection connection;
|
||||
private final TableName tableName;
|
||||
private volatile Configuration configuration;
|
||||
private TableConfiguration tableConfiguration;
|
||||
private ConnectionConfiguration connConfiguration;
|
||||
protected BufferedMutatorImpl mutator;
|
||||
private boolean autoFlush = true;
|
||||
private boolean closed = false;
|
||||
|
@ -298,7 +298,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
*/
|
||||
@InterfaceAudience.Private
|
||||
public HTable(TableName tableName, final ClusterConnection connection,
|
||||
final TableConfiguration tableConfig,
|
||||
final ConnectionConfiguration tableConfig,
|
||||
final RpcRetryingCallerFactory rpcCallerFactory,
|
||||
final RpcControllerFactory rpcControllerFactory,
|
||||
final ExecutorService pool) throws IOException {
|
||||
|
@ -309,7 +309,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
this.cleanupConnectionOnClose = false;
|
||||
this.connection = connection;
|
||||
this.configuration = connection.getConfiguration();
|
||||
this.tableConfiguration = tableConfig;
|
||||
this.connConfiguration = tableConfig;
|
||||
this.pool = pool;
|
||||
if (pool == null) {
|
||||
this.pool = getDefaultExecutor(this.configuration);
|
||||
|
@ -332,7 +332,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException {
|
||||
connection = conn;
|
||||
tableName = params.getTableName();
|
||||
tableConfiguration = new TableConfiguration(connection.getConfiguration());
|
||||
connConfiguration = new ConnectionConfiguration(connection.getConfiguration());
|
||||
cleanupPoolOnClose = false;
|
||||
cleanupConnectionOnClose = false;
|
||||
// used from tests, don't trust the connection is real
|
||||
|
@ -350,14 +350,14 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
* setup this HTable's parameter based on the passed configuration
|
||||
*/
|
||||
private void finishSetup() throws IOException {
|
||||
if (tableConfiguration == null) {
|
||||
tableConfiguration = new TableConfiguration(configuration);
|
||||
if (connConfiguration == null) {
|
||||
connConfiguration = new ConnectionConfiguration(configuration);
|
||||
}
|
||||
|
||||
this.operationTimeout = tableName.isSystemTable() ?
|
||||
tableConfiguration.getMetaOperationTimeout() : tableConfiguration.getOperationTimeout();
|
||||
this.scannerCaching = tableConfiguration.getScannerCaching();
|
||||
this.scannerMaxResultSize = tableConfiguration.getScannerMaxResultSize();
|
||||
connConfiguration.getMetaOperationTimeout() : connConfiguration.getOperationTimeout();
|
||||
this.scannerCaching = connConfiguration.getScannerCaching();
|
||||
this.scannerMaxResultSize = connConfiguration.getScannerMaxResultSize();
|
||||
if (this.rpcCallerFactory == null) {
|
||||
this.rpcCallerFactory = connection.getNewRpcRetryingCallerFactory(configuration);
|
||||
}
|
||||
|
@ -570,23 +570,14 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
*/
|
||||
@Override
|
||||
public HTableDescriptor getTableDescriptor() throws IOException {
|
||||
HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection,
|
||||
rpcCallerFactory, operationTimeout);
|
||||
HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory,
|
||||
rpcControllerFactory, operationTimeout);
|
||||
if (htd != null) {
|
||||
return new UnmodifyableHTableDescriptor(htd);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private <V> V executeMasterCallable(MasterCallable<V> callable) throws IOException {
|
||||
RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
|
||||
try {
|
||||
return caller.callWithRetries(callable, operationTimeout);
|
||||
} finally {
|
||||
callable.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* To be removed in 2.0.0.
|
||||
* @deprecated Since 1.1.0. Use {@link RegionLocator#getStartEndKeys()} instead
|
||||
|
@ -786,22 +777,22 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
if (scan.isSmall()) {
|
||||
return new ClientSmallReversedScanner(getConfiguration(), scan, getName(),
|
||||
this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
|
||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
} else {
|
||||
return new ReversedClientScanner(getConfiguration(), scan, getName(),
|
||||
this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
|
||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
}
|
||||
}
|
||||
|
||||
if (scan.isSmall()) {
|
||||
return new ClientSmallScanner(getConfiguration(), scan, getName(),
|
||||
this.connection, this.rpcCallerFactory, this.rpcControllerFactory,
|
||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
} else {
|
||||
return new ClientScanner(getConfiguration(), scan, getName(), this.connection,
|
||||
this.rpcCallerFactory, this.rpcControllerFactory,
|
||||
pool, tableConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -873,9 +864,9 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
// Call that takes into account the replica
|
||||
RpcRetryingCallerWithReadReplicas callable = new RpcRetryingCallerWithReadReplicas(
|
||||
rpcControllerFactory, tableName, this.connection, get, pool,
|
||||
tableConfiguration.getRetriesNumber(),
|
||||
connConfiguration.getRetriesNumber(),
|
||||
operationTimeout,
|
||||
tableConfiguration.getPrimaryCallTimeoutMicroSecond());
|
||||
connConfiguration.getPrimaryCallTimeoutMicroSecond());
|
||||
return callable.call();
|
||||
}
|
||||
|
||||
|
@ -1517,7 +1508,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
|
||||
// validate for well-formedness
|
||||
public void validatePut(final Put put) throws IllegalArgumentException {
|
||||
validatePut(put, tableConfiguration.getMaxKeyValueSize());
|
||||
validatePut(put, connConfiguration.getMaxKeyValueSize());
|
||||
}
|
||||
|
||||
// validate for well-formedness
|
||||
|
@ -1579,7 +1570,7 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
@Override
|
||||
public long getWriteBufferSize() {
|
||||
if (mutator == null) {
|
||||
return tableConfiguration.getWriteBufferSize();
|
||||
return connConfiguration.getWriteBufferSize();
|
||||
} else {
|
||||
return mutator.getWriteBufferSize();
|
||||
}
|
||||
|
@ -1927,8 +1918,8 @@ public class HTable implements HTableInterface, RegionLocator {
|
|||
this.mutator = (BufferedMutatorImpl) connection.getBufferedMutator(
|
||||
new BufferedMutatorParams(tableName)
|
||||
.pool(pool)
|
||||
.writeBufferSize(tableConfiguration.getWriteBufferSize())
|
||||
.maxKeyValueSize(tableConfiguration.getMaxKeyValueSize())
|
||||
.writeBufferSize(connConfiguration.getWriteBufferSize())
|
||||
.maxKeyValueSize(connConfiguration.getMaxKeyValueSize())
|
||||
);
|
||||
}
|
||||
return mutator;
|
||||
|
|
|
@ -318,7 +318,7 @@ public abstract class AbstractRpcClient implements RpcClient {
|
|||
public Message callBlockingMethod(Descriptors.MethodDescriptor md, RpcController controller,
|
||||
Message param, Message returnType) throws ServiceException {
|
||||
PayloadCarryingRpcController pcrc;
|
||||
if (controller != null) {
|
||||
if (controller != null && controller instanceof PayloadCarryingRpcController) {
|
||||
pcrc = (PayloadCarryingRpcController) controller;
|
||||
if (!pcrc.hasCallTimeout()) {
|
||||
pcrc.setCallTimeout(channelOperationTimeout);
|
||||
|
|
|
@ -53,7 +53,7 @@ public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcCh
|
|||
RpcCallback<Message> callback) {
|
||||
Message response = null;
|
||||
try {
|
||||
response = callExecService(method, request, responsePrototype);
|
||||
response = callExecService(controller, method, request, responsePrototype);
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Call failed on IOException", ioe);
|
||||
ResponseConverter.setControllerException(controller, ioe);
|
||||
|
@ -70,12 +70,13 @@ public abstract class CoprocessorRpcChannel implements RpcChannel, BlockingRpcCh
|
|||
Message request, Message responsePrototype)
|
||||
throws ServiceException {
|
||||
try {
|
||||
return callExecService(method, request, responsePrototype);
|
||||
return callExecService(controller, method, request, responsePrototype);
|
||||
} catch (IOException ioe) {
|
||||
throw new ServiceException("Error calling method "+method.getFullName(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract Message callExecService(Descriptors.MethodDescriptor method,
|
||||
Message request, Message responsePrototype) throws IOException;
|
||||
protected abstract Message callExecService(RpcController controller,
|
||||
Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
|
||||
throws IOException;
|
||||
}
|
||||
|
|
|
@ -26,12 +26,14 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse;
|
||||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
|
||||
|
@ -45,18 +47,18 @@ import com.google.protobuf.Message;
|
|||
public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
||||
private static final Log LOG = LogFactory.getLog(MasterCoprocessorRpcChannel.class);
|
||||
|
||||
private final HConnection connection;
|
||||
private final ClusterConnection connection;
|
||||
|
||||
public MasterCoprocessorRpcChannel(HConnection conn) {
|
||||
public MasterCoprocessorRpcChannel(ClusterConnection conn) {
|
||||
this.connection = conn;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Message callExecService(Descriptors.MethodDescriptor method,
|
||||
protected Message callExecService(RpcController controller, Descriptors.MethodDescriptor method,
|
||||
Message request, Message responsePrototype)
|
||||
throws IOException {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Call: "+method.getName()+", "+request.toString());
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Call: "+method.getName()+", "+request.toString());
|
||||
}
|
||||
|
||||
final ClientProtos.CoprocessorServiceCall call =
|
||||
|
@ -65,7 +67,10 @@ public class MasterCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
|||
.setServiceName(method.getService().getFullName())
|
||||
.setMethodName(method.getName())
|
||||
.setRequest(request.toByteString()).build();
|
||||
CoprocessorServiceResponse result = ProtobufUtil.execService(connection.getMaster(), call);
|
||||
|
||||
// TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
|
||||
CoprocessorServiceResponse result = ProtobufUtil.execService(controller,
|
||||
connection.getMaster(), call);
|
||||
Message response = null;
|
||||
if (result.getValue().hasValue()) {
|
||||
Message.Builder builder = responsePrototype.newBuilderForType();
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.RegionServerCallable;
|
||||
import org.apache.hadoop.hbase.client.RpcRetryingCallerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -36,6 +37,7 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Provides clients with an RPC connection to call coprocessor endpoint {@link com.google.protobuf.Service}s
|
||||
|
@ -49,27 +51,27 @@ import com.google.protobuf.Message;
|
|||
public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
||||
private static final Log LOG = LogFactory.getLog(RegionCoprocessorRpcChannel.class);
|
||||
|
||||
private final HConnection connection;
|
||||
private final ClusterConnection connection;
|
||||
private final TableName table;
|
||||
private final byte[] row;
|
||||
private byte[] lastRegion;
|
||||
private int operationTimeout;
|
||||
|
||||
private RpcRetryingCallerFactory rpcFactory;
|
||||
private RpcRetryingCallerFactory rpcCallerFactory;
|
||||
private RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
public RegionCoprocessorRpcChannel(HConnection conn, TableName table, byte[] row) {
|
||||
public RegionCoprocessorRpcChannel(ClusterConnection conn, TableName table, byte[] row) {
|
||||
this.connection = conn;
|
||||
this.table = table;
|
||||
this.row = row;
|
||||
this.rpcFactory = RpcRetryingCallerFactory.instantiate(conn.getConfiguration(), null);
|
||||
this.operationTimeout = conn.getConfiguration().getInt(
|
||||
HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
|
||||
HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
|
||||
this.rpcCallerFactory = conn.getRpcRetryingCallerFactory();
|
||||
this.rpcControllerFactory = conn.getRpcControllerFactory();
|
||||
this.operationTimeout = conn.getConnectionConfiguration().getOperationTimeout();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Message callExecService(Descriptors.MethodDescriptor method,
|
||||
Message request, Message responsePrototype)
|
||||
protected Message callExecService(RpcController controller,
|
||||
Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
|
||||
throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Call: "+method.getName()+", "+request.toString());
|
||||
|
@ -79,6 +81,9 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
|||
throw new IllegalArgumentException("Missing row property for remote region location");
|
||||
}
|
||||
|
||||
final RpcController rpcController = controller == null
|
||||
? rpcControllerFactory.newController() : controller;
|
||||
|
||||
final ClientProtos.CoprocessorServiceCall call =
|
||||
ClientProtos.CoprocessorServiceCall.newBuilder()
|
||||
.setRow(ByteStringer.wrap(row))
|
||||
|
@ -87,12 +92,19 @@ public class RegionCoprocessorRpcChannel extends CoprocessorRpcChannel{
|
|||
.setRequest(request.toByteString()).build();
|
||||
RegionServerCallable<CoprocessorServiceResponse> callable =
|
||||
new RegionServerCallable<CoprocessorServiceResponse>(connection, table, row) {
|
||||
@Override
|
||||
public CoprocessorServiceResponse call(int callTimeout) throws Exception {
|
||||
if (rpcController instanceof PayloadCarryingRpcController) {
|
||||
((PayloadCarryingRpcController) rpcController).setPriority(tableName);
|
||||
}
|
||||
if (rpcController instanceof TimeLimitedRpcController) {
|
||||
((TimeLimitedRpcController) rpcController).setCallTimeout(callTimeout);
|
||||
}
|
||||
byte[] regionName = getLocation().getRegionInfo().getRegionName();
|
||||
return ProtobufUtil.execService(getStub(), call, regionName);
|
||||
return ProtobufUtil.execService(rpcController, getStub(), call, regionName);
|
||||
}
|
||||
};
|
||||
CoprocessorServiceResponse result = rpcFactory.<CoprocessorServiceResponse> newCaller()
|
||||
CoprocessorServiceResponse result = rpcCallerFactory.<CoprocessorServiceResponse> newCaller()
|
||||
.callWithRetries(callable, operationTimeout);
|
||||
Message response = null;
|
||||
if (result.getValue().hasValue()) {
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
|
|||
|
||||
import com.google.protobuf.Descriptors;
|
||||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.RpcController;
|
||||
|
||||
/**
|
||||
* Provides clients with an RPC connection to call coprocessor endpoint
|
||||
|
@ -48,8 +49,9 @@ public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel {
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Message callExecService(Descriptors.MethodDescriptor method, Message request,
|
||||
Message responsePrototype) throws IOException {
|
||||
protected Message callExecService(RpcController controller,
|
||||
Descriptors.MethodDescriptor method, Message request, Message responsePrototype)
|
||||
throws IOException {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("Call: " + method.getName() + ", " + request.toString());
|
||||
}
|
||||
|
@ -58,8 +60,10 @@ public class RegionServerCoprocessorRpcChannel extends CoprocessorRpcChannel {
|
|||
.setRow(ByteStringer.wrap(HConstants.EMPTY_BYTE_ARRAY))
|
||||
.setServiceName(method.getService().getFullName()).setMethodName(method.getName())
|
||||
.setRequest(request.toByteString()).build();
|
||||
|
||||
// TODO: Are we retrying here? Does not seem so. We should use RetryingRpcCaller
|
||||
CoprocessorServiceResponse result =
|
||||
ProtobufUtil.execRegionServerService(connection.getClient(serverName), call);
|
||||
ProtobufUtil.execRegionServerService(controller, connection.getClient(serverName), call);
|
||||
Message response = null;
|
||||
if (result.getValue().hasValue()) {
|
||||
Message.Builder builder = responsePrototype.newBuilderForType();
|
||||
|
|
|
@ -166,6 +166,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
|
|||
import com.google.protobuf.Message;
|
||||
import com.google.protobuf.Parser;
|
||||
import com.google.protobuf.RpcChannel;
|
||||
import com.google.protobuf.RpcController;
|
||||
import com.google.protobuf.Service;
|
||||
import com.google.protobuf.ServiceException;
|
||||
import com.google.protobuf.TextFormat;
|
||||
|
@ -1616,21 +1617,22 @@ public final class ProtobufUtil {
|
|||
}
|
||||
}
|
||||
|
||||
public static CoprocessorServiceResponse execService(final ClientService.BlockingInterface client,
|
||||
final CoprocessorServiceCall call, final byte[] regionName) throws IOException {
|
||||
public static CoprocessorServiceResponse execService(final RpcController controller,
|
||||
final ClientService.BlockingInterface client, final CoprocessorServiceCall call,
|
||||
final byte[] regionName) throws IOException {
|
||||
CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
|
||||
.setCall(call).setRegion(
|
||||
RequestConverter.buildRegionSpecifier(REGION_NAME, regionName)).build();
|
||||
try {
|
||||
CoprocessorServiceResponse response =
|
||||
client.execService(null, request);
|
||||
client.execService(controller, request);
|
||||
return response;
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
}
|
||||
}
|
||||
|
||||
public static CoprocessorServiceResponse execService(
|
||||
public static CoprocessorServiceResponse execService(final RpcController controller,
|
||||
final MasterService.BlockingInterface client, final CoprocessorServiceCall call)
|
||||
throws IOException {
|
||||
CoprocessorServiceRequest request = CoprocessorServiceRequest.newBuilder()
|
||||
|
@ -1638,7 +1640,7 @@ public final class ProtobufUtil {
|
|||
RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY)).build();
|
||||
try {
|
||||
CoprocessorServiceResponse response =
|
||||
client.execMasterService(null, request);
|
||||
client.execMasterService(controller, request);
|
||||
return response;
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
|
@ -1653,7 +1655,8 @@ public final class ProtobufUtil {
|
|||
* @throws IOException
|
||||
*/
|
||||
public static CoprocessorServiceResponse execRegionServerService(
|
||||
final ClientService.BlockingInterface client, final CoprocessorServiceCall call)
|
||||
final RpcController controller, final ClientService.BlockingInterface client,
|
||||
final CoprocessorServiceCall call)
|
||||
throws IOException {
|
||||
CoprocessorServiceRequest request =
|
||||
CoprocessorServiceRequest
|
||||
|
@ -1663,7 +1666,7 @@ public final class ProtobufUtil {
|
|||
RequestConverter.buildRegionSpecifier(REGION_NAME, HConstants.EMPTY_BYTE_ARRAY))
|
||||
.build();
|
||||
try {
|
||||
CoprocessorServiceResponse response = client.execRegionServerService(null, request);
|
||||
CoprocessorServiceResponse response = client.execRegionServerService(controller, request);
|
||||
return response;
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
|
@ -1689,13 +1692,13 @@ public final class ProtobufUtil {
|
|||
* @return the retrieved region info
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HRegionInfo getRegionInfo(final AdminService.BlockingInterface admin,
|
||||
final byte[] regionName) throws IOException {
|
||||
public static HRegionInfo getRegionInfo(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, final byte[] regionName) throws IOException {
|
||||
try {
|
||||
GetRegionInfoRequest request =
|
||||
RequestConverter.buildGetRegionInfoRequest(regionName);
|
||||
GetRegionInfoResponse response =
|
||||
admin.getRegionInfo(null, request);
|
||||
admin.getRegionInfo(controller, request);
|
||||
return HRegionInfo.convert(response.getRegionInfo());
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
|
@ -1711,12 +1714,13 @@ public final class ProtobufUtil {
|
|||
* @param transitionInZK
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void closeRegion(final AdminService.BlockingInterface admin,
|
||||
final ServerName server, final byte[] regionName, final boolean transitionInZK) throws IOException {
|
||||
public static void closeRegion(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, final ServerName server, final byte[] regionName,
|
||||
final boolean transitionInZK) throws IOException {
|
||||
CloseRegionRequest closeRegionRequest =
|
||||
RequestConverter.buildCloseRegionRequest(server, regionName, transitionInZK);
|
||||
try {
|
||||
admin.closeRegion(null, closeRegionRequest);
|
||||
admin.closeRegion(controller, closeRegionRequest);
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
}
|
||||
|
@ -1732,7 +1736,8 @@ public final class ProtobufUtil {
|
|||
* @return true if the region is closed
|
||||
* @throws IOException
|
||||
*/
|
||||
public static boolean closeRegion(final AdminService.BlockingInterface admin,
|
||||
public static boolean closeRegion(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin,
|
||||
final ServerName server,
|
||||
final byte[] regionName,
|
||||
final int versionOfClosingNode, final ServerName destinationServer,
|
||||
|
@ -1741,7 +1746,7 @@ public final class ProtobufUtil {
|
|||
RequestConverter.buildCloseRegionRequest(server,
|
||||
regionName, versionOfClosingNode, destinationServer, transitionInZK);
|
||||
try {
|
||||
CloseRegionResponse response = admin.closeRegion(null, closeRegionRequest);
|
||||
CloseRegionResponse response = admin.closeRegion(controller, closeRegionRequest);
|
||||
return ResponseConverter.isClosed(response);
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
|
@ -1756,14 +1761,14 @@ public final class ProtobufUtil {
|
|||
* @param regionInfo
|
||||
*
|
||||
*/
|
||||
public static void warmupRegion(final AdminService.BlockingInterface admin,
|
||||
final HRegionInfo regionInfo) throws IOException {
|
||||
public static void warmupRegion(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, final HRegionInfo regionInfo) throws IOException {
|
||||
|
||||
try {
|
||||
WarmupRegionRequest warmupRegionRequest =
|
||||
RequestConverter.buildWarmupRegionRequest(regionInfo);
|
||||
|
||||
admin.warmupRegion(null, warmupRegionRequest);
|
||||
admin.warmupRegion(controller, warmupRegionRequest);
|
||||
} catch (ServiceException e) {
|
||||
throw getRemoteException(e);
|
||||
}
|
||||
|
@ -1775,18 +1780,18 @@ public final class ProtobufUtil {
|
|||
* @param region
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void openRegion(final AdminService.BlockingInterface admin,
|
||||
ServerName server, final HRegionInfo region) throws IOException {
|
||||
public static void openRegion(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, ServerName server, final HRegionInfo region)
|
||||
throws IOException {
|
||||
OpenRegionRequest request =
|
||||
RequestConverter.buildOpenRegionRequest(server, region, -1, null, null);
|
||||
try {
|
||||
admin.openRegion(null, request);
|
||||
admin.openRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* A helper to get the all the online regions on a region
|
||||
* server using admin protocol.
|
||||
|
@ -1797,10 +1802,21 @@ public final class ProtobufUtil {
|
|||
*/
|
||||
public static List<HRegionInfo> getOnlineRegions(final AdminService.BlockingInterface admin)
|
||||
throws IOException {
|
||||
return getOnlineRegions(null, admin);
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper to get the all the online regions on a region
|
||||
* server using admin protocol.
|
||||
* @return a list of online region info
|
||||
*/
|
||||
public static List<HRegionInfo> getOnlineRegions(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin)
|
||||
throws IOException {
|
||||
GetOnlineRegionRequest request = RequestConverter.buildGetOnlineRegionRequest();
|
||||
GetOnlineRegionResponse response = null;
|
||||
try {
|
||||
response = admin.getOnlineRegion(null, request);
|
||||
response = admin.getOnlineRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
}
|
||||
|
@ -1824,16 +1840,14 @@ public final class ProtobufUtil {
|
|||
|
||||
/**
|
||||
* A helper to get the info of a region server using admin protocol.
|
||||
*
|
||||
* @param admin
|
||||
* @return the server name
|
||||
* @throws IOException
|
||||
*/
|
||||
public static ServerInfo getServerInfo(final AdminService.BlockingInterface admin)
|
||||
public static ServerInfo getServerInfo(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin)
|
||||
throws IOException {
|
||||
GetServerInfoRequest request = RequestConverter.buildGetServerInfoRequest();
|
||||
try {
|
||||
GetServerInfoResponse response = admin.getServerInfo(null, request);
|
||||
GetServerInfoResponse response = admin.getServerInfo(controller, request);
|
||||
return response.getServerInfo();
|
||||
} catch (ServiceException se) {
|
||||
throw getRemoteException(se);
|
||||
|
@ -1844,19 +1858,27 @@ public final class ProtobufUtil {
|
|||
* A helper to get the list of files of a column family
|
||||
* on a given region using admin protocol.
|
||||
*
|
||||
* @param admin
|
||||
* @param regionName
|
||||
* @param family
|
||||
* @return the list of store files
|
||||
* @throws IOException
|
||||
*/
|
||||
public static List<String> getStoreFiles(final AdminService.BlockingInterface admin,
|
||||
final byte[] regionName, final byte[] family)
|
||||
throws IOException {
|
||||
return getStoreFiles(null, admin, regionName, family);
|
||||
}
|
||||
|
||||
/**
|
||||
* A helper to get the list of files of a column family
|
||||
* on a given region using admin protocol.
|
||||
*
|
||||
* @return the list of store files
|
||||
*/
|
||||
public static List<String> getStoreFiles(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, final byte[] regionName, final byte[] family)
|
||||
throws IOException {
|
||||
GetStoreFileRequest request =
|
||||
RequestConverter.buildGetStoreFileRequest(regionName, family);
|
||||
try {
|
||||
GetStoreFileResponse response = admin.getStoreFile(null, request);
|
||||
GetStoreFileResponse response = admin.getStoreFile(controller, request);
|
||||
return response.getStoreFileList();
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
|
@ -1871,12 +1893,13 @@ public final class ProtobufUtil {
|
|||
* @param splitPoint
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void split(final AdminService.BlockingInterface admin,
|
||||
final HRegionInfo hri, byte[] splitPoint) throws IOException {
|
||||
public static void split(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin, final HRegionInfo hri, byte[] splitPoint)
|
||||
throws IOException {
|
||||
SplitRegionRequest request =
|
||||
RequestConverter.buildSplitRegionRequest(hri.getRegionName(), splitPoint);
|
||||
try {
|
||||
admin.splitRegion(null, request);
|
||||
admin.splitRegion(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -1893,7 +1916,8 @@ public final class ProtobufUtil {
|
|||
* @param user effective user
|
||||
* @throws IOException
|
||||
*/
|
||||
public static void mergeRegions(final AdminService.BlockingInterface admin,
|
||||
public static void mergeRegions(final RpcController controller,
|
||||
final AdminService.BlockingInterface admin,
|
||||
final HRegionInfo region_a, final HRegionInfo region_b,
|
||||
final boolean forcible, final User user) throws IOException {
|
||||
final MergeRegionsRequest request = RequestConverter.buildMergeRegionsRequest(
|
||||
|
@ -1903,7 +1927,7 @@ public final class ProtobufUtil {
|
|||
user.getUGI().doAs(new PrivilegedExceptionAction<Void>() {
|
||||
@Override
|
||||
public Void run() throws Exception {
|
||||
admin.mergeRegions(null, request);
|
||||
admin.mergeRegions(controller, request);
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
@ -1914,7 +1938,7 @@ public final class ProtobufUtil {
|
|||
}
|
||||
} else {
|
||||
try {
|
||||
admin.mergeRegions(null, request);
|
||||
admin.mergeRegions(controller, request);
|
||||
} catch (ServiceException se) {
|
||||
throw ProtobufUtil.getRemoteException(se);
|
||||
}
|
||||
|
@ -2188,8 +2212,9 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be granted
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void grant(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, Permission.Action... actions) throws ServiceException {
|
||||
public static void grant(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
for (Permission.Action a : actions) {
|
||||
|
@ -2198,7 +2223,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.GrantRequest request = RequestConverter.
|
||||
buildGrantRequest(userShortName, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.grant(null, request);
|
||||
protocol.grant(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2215,9 +2240,9 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be granted
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void grant(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, TableName tableName, byte[] f, byte[] q,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
public static void grant(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName, TableName tableName,
|
||||
byte[] f, byte[] q, Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
for (Permission.Action a : actions) {
|
||||
|
@ -2226,7 +2251,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.GrantRequest request = RequestConverter.
|
||||
buildGrantRequest(userShortName, tableName, f, q, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.grant(null, request);
|
||||
protocol.grant(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2239,8 +2264,8 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be granted
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void grant(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, String namespace,
|
||||
public static void grant(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName, String namespace,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
|
@ -2250,7 +2275,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.GrantRequest request = RequestConverter.
|
||||
buildGrantRequest(userShortName, namespace, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.grant(null, request);
|
||||
protocol.grant(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2263,8 +2288,9 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be revoked
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void revoke(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, Permission.Action... actions) throws ServiceException {
|
||||
public static void revoke(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
for (Permission.Action a : actions) {
|
||||
|
@ -2273,7 +2299,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.RevokeRequest request = RequestConverter.
|
||||
buildRevokeRequest(userShortName, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.revoke(null, request);
|
||||
protocol.revoke(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2290,9 +2316,9 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be revoked
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void revoke(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, TableName tableName, byte[] f, byte[] q,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
public static void revoke(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName, TableName tableName,
|
||||
byte[] f, byte[] q, Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
for (Permission.Action a : actions) {
|
||||
|
@ -2301,7 +2327,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.RevokeRequest request = RequestConverter.
|
||||
buildRevokeRequest(userShortName, tableName, f, q, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.revoke(null, request);
|
||||
protocol.revoke(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2315,8 +2341,8 @@ public final class ProtobufUtil {
|
|||
* @param actions the permissions to be revoked
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static void revoke(AccessControlService.BlockingInterface protocol,
|
||||
String userShortName, String namespace,
|
||||
public static void revoke(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol, String userShortName, String namespace,
|
||||
Permission.Action... actions) throws ServiceException {
|
||||
List<AccessControlProtos.Permission.Action> permActions =
|
||||
Lists.newArrayListWithCapacity(actions.length);
|
||||
|
@ -2326,7 +2352,7 @@ public final class ProtobufUtil {
|
|||
AccessControlProtos.RevokeRequest request = RequestConverter.
|
||||
buildRevokeRequest(userShortName, namespace, permActions.toArray(
|
||||
new AccessControlProtos.Permission.Action[actions.length]));
|
||||
protocol.revoke(null, request);
|
||||
protocol.revoke(controller, request);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2337,14 +2363,14 @@ public final class ProtobufUtil {
|
|||
* @param protocol the AccessControlService protocol proxy
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static List<UserPermission> getUserPermissions(
|
||||
public static List<UserPermission> getUserPermissions(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol) throws ServiceException {
|
||||
AccessControlProtos.GetUserPermissionsRequest.Builder builder =
|
||||
AccessControlProtos.GetUserPermissionsRequest.newBuilder();
|
||||
builder.setType(AccessControlProtos.Permission.Type.Global);
|
||||
AccessControlProtos.GetUserPermissionsRequest request = builder.build();
|
||||
AccessControlProtos.GetUserPermissionsResponse response =
|
||||
protocol.getUserPermissions(null, request);
|
||||
protocol.getUserPermissions(controller, request);
|
||||
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
|
||||
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
|
||||
perms.add(ProtobufUtil.toUserPermission(perm));
|
||||
|
@ -2361,7 +2387,7 @@ public final class ProtobufUtil {
|
|||
* @param t optional table name
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static List<UserPermission> getUserPermissions(
|
||||
public static List<UserPermission> getUserPermissions(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol,
|
||||
TableName t) throws ServiceException {
|
||||
AccessControlProtos.GetUserPermissionsRequest.Builder builder =
|
||||
|
@ -2372,7 +2398,7 @@ public final class ProtobufUtil {
|
|||
builder.setType(AccessControlProtos.Permission.Type.Table);
|
||||
AccessControlProtos.GetUserPermissionsRequest request = builder.build();
|
||||
AccessControlProtos.GetUserPermissionsResponse response =
|
||||
protocol.getUserPermissions(null, request);
|
||||
protocol.getUserPermissions(controller, request);
|
||||
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
|
||||
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
|
||||
perms.add(ProtobufUtil.toUserPermission(perm));
|
||||
|
@ -2389,7 +2415,7 @@ public final class ProtobufUtil {
|
|||
* @param namespace name of the namespace
|
||||
* @throws ServiceException
|
||||
*/
|
||||
public static List<UserPermission> getUserPermissions(
|
||||
public static List<UserPermission> getUserPermissions(RpcController controller,
|
||||
AccessControlService.BlockingInterface protocol,
|
||||
byte[] namespace) throws ServiceException {
|
||||
AccessControlProtos.GetUserPermissionsRequest.Builder builder =
|
||||
|
@ -2400,7 +2426,7 @@ public final class ProtobufUtil {
|
|||
builder.setType(AccessControlProtos.Permission.Type.Namespace);
|
||||
AccessControlProtos.GetUserPermissionsRequest request = builder.build();
|
||||
AccessControlProtos.GetUserPermissionsResponse response =
|
||||
protocol.getUserPermissions(null, request);
|
||||
protocol.getUserPermissions(controller, request);
|
||||
List<UserPermission> perms = new ArrayList<UserPermission>(response.getUserPermissionCount());
|
||||
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
|
||||
perms.add(ProtobufUtil.toUserPermission(perm));
|
||||
|
|
|
@ -32,11 +32,13 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
|||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.client.Admin;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.client.security.SecurityCapability;
|
||||
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService.BlockingInterface;
|
||||
|
@ -94,9 +96,12 @@ public class AccessControlClient {
|
|||
public static void grant(final Connection connection, final TableName tableName,
|
||||
final String userName, final byte[] family, final byte[] qual,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
controller.setPriority(tableName);
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, tableName, family, qual,
|
||||
actions);
|
||||
ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, tableName,
|
||||
family, qual, actions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -110,8 +115,12 @@ public class AccessControlClient {
|
|||
*/
|
||||
public static void grant(final Connection connection, final String namespace,
|
||||
final String userName, final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, namespace,
|
||||
actions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -121,8 +130,10 @@ public class AccessControlClient {
|
|||
*/
|
||||
public static void grant(final Connection connection, final String userName,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.grant(getAccessControlServiceStub(table), userName, actions);
|
||||
ProtobufUtil.grant(controller, getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -146,9 +157,12 @@ public class AccessControlClient {
|
|||
public static void revoke(final Connection connection, final TableName tableName,
|
||||
final String username, final byte[] family, final byte[] qualifier,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
controller.setPriority(tableName);
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), username, tableName, family,
|
||||
qualifier, actions);
|
||||
ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), username, tableName,
|
||||
family, qualifier, actions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -162,8 +176,11 @@ public class AccessControlClient {
|
|||
*/
|
||||
public static void revoke(final Connection connection, final String namespace,
|
||||
final String userName, final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, namespace, actions);
|
||||
ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, namespace,
|
||||
actions);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -173,10 +190,11 @@ public class AccessControlClient {
|
|||
*/
|
||||
public static void revoke(final Connection connection, final String userName,
|
||||
final Permission.Action... actions) throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
ProtobufUtil.revoke(getAccessControlServiceStub(table), userName, actions);
|
||||
ProtobufUtil.revoke(controller, getAccessControlServiceStub(table), userName, actions);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -188,6 +206,8 @@ public class AccessControlClient {
|
|||
*/
|
||||
public static List<UserPermission> getUserPermissions(Connection connection, String tableRegex)
|
||||
throws Throwable {
|
||||
PayloadCarryingRpcController controller
|
||||
= ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
List<UserPermission> permList = new ArrayList<UserPermission>();
|
||||
try (Table table = connection.getTable(ACL_TABLE_NAME)) {
|
||||
try (Admin admin = connection.getAdmin()) {
|
||||
|
@ -196,14 +216,16 @@ public class AccessControlClient {
|
|||
AccessControlProtos.AccessControlService.newBlockingStub(service);
|
||||
HTableDescriptor[] htds = null;
|
||||
if (tableRegex == null || tableRegex.isEmpty()) {
|
||||
permList = ProtobufUtil.getUserPermissions(protocol);
|
||||
permList = ProtobufUtil.getUserPermissions(controller, protocol);
|
||||
} else if (tableRegex.charAt(0) == '@') {
|
||||
String namespace = tableRegex.substring(1);
|
||||
permList = ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(namespace));
|
||||
permList = ProtobufUtil.getUserPermissions(controller, protocol,
|
||||
Bytes.toBytes(namespace));
|
||||
} else {
|
||||
htds = admin.listTables(Pattern.compile(tableRegex), true);
|
||||
for (HTableDescriptor hd : htds) {
|
||||
permList.addAll(ProtobufUtil.getUserPermissions(protocol, hd.getTableName()));
|
||||
permList.addAll(ProtobufUtil.getUserPermissions(controller, protocol,
|
||||
hd.getTableName()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,11 +35,14 @@ import org.apache.hadoop.hbase.HRegionInfo;
|
|||
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.HConnection;
|
||||
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.exceptions.DeserializationException;
|
||||
import org.apache.hadoop.hbase.ipc.FailedServerException;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
|
||||
|
@ -286,7 +289,7 @@ public class MetaTableLocator {
|
|||
} catch (RegionServerStoppedException e) {
|
||||
// Pass -- server name sends us to a server that is dying or already dead.
|
||||
}
|
||||
return (service != null) && verifyRegionLocation(service,
|
||||
return (service != null) && verifyRegionLocation(hConnection, service,
|
||||
getMetaRegionLocation(zkw, replicaId), RegionReplicaUtil.getRegionInfoForReplica(
|
||||
HRegionInfo.FIRST_META_REGIONINFO, replicaId).getRegionName());
|
||||
}
|
||||
|
@ -306,17 +309,22 @@ public class MetaTableLocator {
|
|||
// rather than have to pass it in. Its made awkward by the fact that the
|
||||
// HRI is likely a proxy against remote server so the getServerName needs
|
||||
// to be fixed to go to a local method or to a cache before we can do this.
|
||||
private boolean verifyRegionLocation(AdminService.BlockingInterface hostingServer,
|
||||
final ServerName address, final byte [] regionName)
|
||||
private boolean verifyRegionLocation(final Connection connection,
|
||||
AdminService.BlockingInterface hostingServer, final ServerName address,
|
||||
final byte [] regionName)
|
||||
throws IOException {
|
||||
if (hostingServer == null) {
|
||||
LOG.info("Passed hostingServer is null");
|
||||
return false;
|
||||
}
|
||||
Throwable t;
|
||||
PayloadCarryingRpcController controller = null;
|
||||
if (connection instanceof ClusterConnection) {
|
||||
controller = ((ClusterConnection) connection).getRpcControllerFactory().newController();
|
||||
}
|
||||
try {
|
||||
// Try and get regioninfo from the hosting server.
|
||||
return ProtobufUtil.getRegionInfo(hostingServer, regionName) != null;
|
||||
return ProtobufUtil.getRegionInfo(controller, hostingServer, regionName) != null;
|
||||
} catch (ConnectException e) {
|
||||
t = e;
|
||||
} catch (RetriesExhaustedException e) {
|
||||
|
|
|
@ -29,6 +29,8 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
|||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
|
||||
|
@ -77,24 +79,32 @@ public class TestSnapshotFromAdmin {
|
|||
// setup the conf to match the expected properties
|
||||
conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, numRetries);
|
||||
conf.setLong("hbase.client.pause", pauseTime);
|
||||
|
||||
// mock the master admin to our mock
|
||||
MasterKeepAliveConnection mockMaster = Mockito.mock(MasterKeepAliveConnection.class);
|
||||
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(mockMaster);
|
||||
// we need a real retrying caller
|
||||
RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
|
||||
RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
|
||||
Mockito.when(controllerFactory.newController()).thenReturn(
|
||||
Mockito.mock(PayloadCarryingRpcController.class));
|
||||
Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
|
||||
Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
|
||||
// set the max wait time for the snapshot to complete
|
||||
SnapshotResponse response = SnapshotResponse.newBuilder()
|
||||
.setExpectedTimeout(maxWaitTime)
|
||||
.build();
|
||||
Mockito
|
||||
.when(
|
||||
mockMaster.snapshot((RpcController) Mockito.isNull(),
|
||||
mockMaster.snapshot((RpcController) Mockito.any(),
|
||||
Mockito.any(SnapshotRequest.class))).thenReturn(response);
|
||||
// setup the response
|
||||
IsSnapshotDoneResponse.Builder builder = IsSnapshotDoneResponse.newBuilder();
|
||||
builder.setDone(false);
|
||||
// first five times, we return false, last we get success
|
||||
Mockito.when(
|
||||
mockMaster.isSnapshotDone((RpcController) Mockito.isNull(),
|
||||
mockMaster.isSnapshotDone((RpcController) Mockito.any(),
|
||||
Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(builder.build(), builder.build(),
|
||||
builder.build(), builder.build(), builder.build(), builder.setDone(true).build());
|
||||
|
||||
|
@ -122,6 +132,13 @@ public class TestSnapshotFromAdmin {
|
|||
.mock(ConnectionManager.HConnectionImplementation.class);
|
||||
Configuration conf = HBaseConfiguration.create();
|
||||
Mockito.when(mockConnection.getConfiguration()).thenReturn(conf);
|
||||
// we need a real retrying caller
|
||||
RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
|
||||
RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
|
||||
Mockito.when(controllerFactory.newController()).thenReturn(
|
||||
Mockito.mock(PayloadCarryingRpcController.class));
|
||||
Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
|
||||
Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
|
||||
Admin admin = new HBaseAdmin(mockConnection);
|
||||
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
|
||||
// check that invalid snapshot names fail
|
||||
|
@ -141,11 +158,11 @@ public class TestSnapshotFromAdmin {
|
|||
Mockito.when(mockConnection.getKeepAliveMasterService()).thenReturn(master);
|
||||
SnapshotResponse response = SnapshotResponse.newBuilder().setExpectedTimeout(0).build();
|
||||
Mockito.when(
|
||||
master.snapshot((RpcController) Mockito.isNull(), Mockito.any(SnapshotRequest.class)))
|
||||
master.snapshot((RpcController) Mockito.any(), Mockito.any(SnapshotRequest.class)))
|
||||
.thenReturn(response);
|
||||
IsSnapshotDoneResponse doneResponse = IsSnapshotDoneResponse.newBuilder().setDone(true).build();
|
||||
Mockito.when(
|
||||
master.isSnapshotDone((RpcController) Mockito.isNull(),
|
||||
master.isSnapshotDone((RpcController) Mockito.any(),
|
||||
Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
|
||||
|
||||
// make sure that we can use valid names
|
||||
|
|
|
@ -279,7 +279,7 @@ public class DistributedHBaseCluster extends HBaseCluster {
|
|||
|
||||
AdminProtos.AdminService.BlockingInterface client =
|
||||
((ClusterConnection)this.connection).getAdmin(regionLoc.getServerName());
|
||||
ServerInfo info = ProtobufUtil.getServerInfo(client);
|
||||
ServerInfo info = ProtobufUtil.getServerInfo(null, client);
|
||||
return ProtobufUtil.toServerName(info.getServerName());
|
||||
}
|
||||
|
||||
|
|
|
@ -42,7 +42,7 @@ org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
|
|||
<%java return; %>
|
||||
</%if>
|
||||
<%java>
|
||||
ServerInfo serverInfo = ProtobufUtil.getServerInfo(regionServer.getRSRpcServices());
|
||||
ServerInfo serverInfo = ProtobufUtil.getServerInfo(null, regionServer.getRSRpcServices());
|
||||
ServerName serverName = ProtobufUtil.toServerName(serverInfo.getServerName());
|
||||
List<HRegionInfo> onlineRegions = ProtobufUtil.getOnlineRegions(regionServer.getRSRpcServices());
|
||||
MasterAddressTracker masterAddressTracker = regionServer.getMasterAddressTracker();
|
||||
|
|
|
@ -22,7 +22,9 @@ import java.nio.channels.ClosedChannelException;
|
|||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.hbase.CellScanner;
|
||||
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hbase.classification.InterfaceStability;
|
||||
import org.apache.hadoop.hbase.ipc.RpcServer.Call;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
|
||||
import org.apache.hadoop.hbase.util.Pair;
|
||||
|
@ -38,7 +40,8 @@ import com.google.protobuf.Message;
|
|||
* {@link RpcScheduler}. Call {@link #run()} to actually execute the contained
|
||||
* RpcServer.Call
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class CallRunner {
|
||||
private static final Log LOG = LogFactory.getLog(CallRunner.class);
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ public abstract class RpcScheduler {
|
|||
"hbase.ipc.server.priority.max.callqueue.length";
|
||||
|
||||
/** Exposes runtime information of a {@code RpcServer} that a {@code RpcScheduler} may need. */
|
||||
static abstract class Context {
|
||||
public static abstract class Context {
|
||||
public abstract InetSocketAddress getListenerAddress();
|
||||
}
|
||||
|
||||
|
|
|
@ -291,7 +291,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
|||
* Datastructure that holds all necessary to a method invocation and then afterward, carries
|
||||
* the result.
|
||||
*/
|
||||
class Call implements RpcCallContext {
|
||||
@InterfaceAudience.LimitedPrivate({HBaseInterfaceAudience.COPROC, HBaseInterfaceAudience.PHOENIX})
|
||||
@InterfaceStability.Evolving
|
||||
public class Call implements RpcCallContext {
|
||||
protected int id; // the client's call id
|
||||
protected BlockingService service;
|
||||
protected MethodDescriptor md;
|
||||
|
@ -370,6 +372,14 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
|
|||
return this.header;
|
||||
}
|
||||
|
||||
public boolean hasPriority() {
|
||||
return this.header.hasPriority();
|
||||
}
|
||||
|
||||
public int getPriority() {
|
||||
return this.header.getPriority();
|
||||
}
|
||||
|
||||
/*
|
||||
* Short string representation without param info because param itself could be huge depends on
|
||||
* the payload of a command
|
||||
|
|
|
@ -695,7 +695,7 @@ public class RegionPlacementMaintainer {
|
|||
UpdateFavoredNodesResponse updateFavoredNodesResponse =
|
||||
currentRegionServer.updateFavoredNodes(null, request);
|
||||
LOG.info("Region server " +
|
||||
ProtobufUtil.getServerInfo(currentRegionServer).getServerName() +
|
||||
ProtobufUtil.getServerInfo(null, currentRegionServer).getServerName() +
|
||||
" has updated " + updateFavoredNodesResponse.getResponse() + " / " +
|
||||
singleServerPlan.getAssignmentMap().size() +
|
||||
" regions with the assignment plan");
|
||||
|
|
|
@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
|
||||
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
|
||||
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
|
||||
|
@ -162,6 +164,7 @@ public class ServerManager {
|
|||
private final long warningSkew;
|
||||
|
||||
private final RetryCounterFactory pingRetryCounterFactory;
|
||||
private final RpcControllerFactory rpcControllerFactory;
|
||||
|
||||
/**
|
||||
* Set of region servers which are dead but not processed immediately. If one
|
||||
|
@ -226,6 +229,9 @@ public class ServerManager {
|
|||
int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
|
||||
"hbase.master.ping.server.retry.sleep.interval", 100));
|
||||
this.pingRetryCounterFactory = new RetryCounterFactory(pingMaxAttempts, pingSleepInterval);
|
||||
this.rpcControllerFactory = this.connection == null
|
||||
? null
|
||||
: connection.getRpcControllerFactory();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -793,6 +799,10 @@ public class ServerManager {
|
|||
}
|
||||
}
|
||||
|
||||
private PayloadCarryingRpcController newRpcController() {
|
||||
return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends an CLOSE RPC to the specified server to close the specified region.
|
||||
* <p>
|
||||
|
@ -817,7 +827,8 @@ public class ServerManager {
|
|||
region.getRegionNameAsString() +
|
||||
" failed because no RPC connection found to this server");
|
||||
}
|
||||
return ProtobufUtil.closeRegion(admin, server, region.getRegionName(),
|
||||
PayloadCarryingRpcController controller = newRpcController();
|
||||
return ProtobufUtil.closeRegion(controller, admin, server, region.getRegionName(),
|
||||
versionOfClosingNode, dest, transitionInZK);
|
||||
}
|
||||
|
||||
|
@ -839,7 +850,8 @@ public class ServerManager {
|
|||
if (server == null) return;
|
||||
try {
|
||||
AdminService.BlockingInterface admin = getRsAdmin(server);
|
||||
ProtobufUtil.warmupRegion(admin, region);
|
||||
PayloadCarryingRpcController controller = newRpcController();
|
||||
ProtobufUtil.warmupRegion(controller, admin, region);
|
||||
} catch (IOException e) {
|
||||
LOG.error("Received exception in RPC for warmup server:" +
|
||||
server + "region: " + region +
|
||||
|
@ -854,8 +866,9 @@ public class ServerManager {
|
|||
public static void closeRegionSilentlyAndWait(ClusterConnection connection,
|
||||
ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
|
||||
AdminService.BlockingInterface rs = connection.getAdmin(server);
|
||||
PayloadCarryingRpcController controller = connection.getRpcControllerFactory().newController();
|
||||
try {
|
||||
ProtobufUtil.closeRegion(rs, server, region.getRegionName(), false);
|
||||
ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName(), false);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
|
||||
}
|
||||
|
@ -863,12 +876,13 @@ public class ServerManager {
|
|||
while (System.currentTimeMillis() < expiration) {
|
||||
try {
|
||||
HRegionInfo rsRegion =
|
||||
ProtobufUtil.getRegionInfo(rs, region.getRegionName());
|
||||
ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
|
||||
if (rsRegion == null) return;
|
||||
} catch (IOException ioe) {
|
||||
if (ioe instanceof NotServingRegionException) // no need to retry again
|
||||
return;
|
||||
LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), ioe);
|
||||
LOG.warn("Exception when retrieving regioninfo from: "
|
||||
+ region.getRegionNameAsString(), ioe);
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
@ -903,7 +917,8 @@ public class ServerManager {
|
|||
+ region_b.getRegionNameAsString()
|
||||
+ " failed because no RPC connection found to this server");
|
||||
}
|
||||
ProtobufUtil.mergeRegions(admin, region_a, region_b, forcible, user);
|
||||
PayloadCarryingRpcController controller = newRpcController();
|
||||
ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible, user);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -912,6 +927,7 @@ public class ServerManager {
|
|||
public boolean isServerReachable(ServerName server) {
|
||||
if (server == null) throw new NullPointerException("Passed server is null");
|
||||
|
||||
|
||||
RetryCounter retryCounter = pingRetryCounterFactory.create();
|
||||
while (retryCounter.shouldRetry()) {
|
||||
synchronized (this.onlineServers) {
|
||||
|
@ -920,9 +936,10 @@ public class ServerManager {
|
|||
}
|
||||
}
|
||||
try {
|
||||
PayloadCarryingRpcController controller = newRpcController();
|
||||
AdminService.BlockingInterface admin = getRsAdmin(server);
|
||||
if (admin != null) {
|
||||
ServerInfo info = ProtobufUtil.getServerInfo(admin);
|
||||
ServerInfo info = ProtobufUtil.getServerInfo(controller, admin);
|
||||
return info != null && info.hasServerName()
|
||||
&& server.getStartcode() == info.getServerName().getStartCode();
|
||||
}
|
||||
|
|
|
@ -693,8 +693,8 @@ public class MiniHBaseCluster extends HBaseCluster {
|
|||
int count = 0;
|
||||
for (JVMClusterUtil.RegionServerThread rst: getRegionServerThreads()) {
|
||||
HRegionServer hrs = rst.getRegionServer();
|
||||
Region metaRegion = hrs.getOnlineRegion(regionName);
|
||||
if (metaRegion != null) {
|
||||
Region region = hrs.getOnlineRegion(regionName);
|
||||
if (region != null) {
|
||||
index = count;
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -86,7 +86,7 @@ public class TestGlobalMemStoreSize {
|
|||
for (HRegionServer server : getOnlineRegionServers()) {
|
||||
long globalMemStoreSize = 0;
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
globalMemStoreSize +=
|
||||
server.getFromOnlineRegions(regionInfo.getEncodedName()).
|
||||
getMemstoreSize();
|
||||
|
@ -102,7 +102,7 @@ public class TestGlobalMemStoreSize {
|
|||
", size=" + server.getRegionServerAccounting().getGlobalMemstoreSize());
|
||||
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
|
||||
flush(r, server);
|
||||
}
|
||||
|
@ -118,7 +118,7 @@ public class TestGlobalMemStoreSize {
|
|||
// If size > 0, see if its because the meta region got edits while
|
||||
// our test was running....
|
||||
for (HRegionInfo regionInfo :
|
||||
ProtobufUtil.getOnlineRegions(server.getRSRpcServices())) {
|
||||
ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) {
|
||||
Region r = server.getFromOnlineRegions(regionInfo.getEncodedName());
|
||||
long l = r.getMemstoreSize();
|
||||
if (l > 0) {
|
||||
|
|
|
@ -39,6 +39,13 @@ import org.apache.hadoop.hbase.client.HTable;
|
|||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.client.Table;
|
||||
import org.apache.hadoop.hbase.ipc.CallRunner;
|
||||
import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
|
||||
import org.apache.hadoop.hbase.ipc.PriorityFunction;
|
||||
import org.apache.hadoop.hbase.ipc.RpcScheduler;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.hbase.regionserver.RSRpcServices;
|
||||
import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
|
||||
import org.apache.hadoop.hbase.testclassification.MediumTests;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
||||
|
@ -565,5 +572,77 @@ public class TestMetaTableAccessor {
|
|||
meta.close();
|
||||
}
|
||||
}
|
||||
|
||||
public static class SpyingRpcSchedulerFactory extends SimpleRpcSchedulerFactory {
|
||||
@Override
|
||||
public RpcScheduler create(Configuration conf, PriorityFunction priority, Abortable server) {
|
||||
final RpcScheduler delegate = super.create(conf, priority, server);
|
||||
return new SpyingRpcScheduler(delegate);
|
||||
}
|
||||
}
|
||||
|
||||
public static class SpyingRpcScheduler extends DelegatingRpcScheduler {
|
||||
long numPriorityCalls = 0;
|
||||
|
||||
public SpyingRpcScheduler(RpcScheduler delegate) {
|
||||
super(delegate);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dispatch(CallRunner task) throws IOException, InterruptedException {
|
||||
int priority = task.getCall().getPriority();
|
||||
|
||||
if (priority > HConstants.QOS_THRESHOLD) {
|
||||
numPriorityCalls++;
|
||||
}
|
||||
return super.dispatch(task);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetaUpdatesGoToPriorityQueue() throws Exception {
|
||||
// This test has to be end-to-end, and do the verification from the server side
|
||||
Configuration c = UTIL.getConfiguration();
|
||||
|
||||
c.set(RSRpcServices.REGION_SERVER_RPC_SCHEDULER_FACTORY_CLASS,
|
||||
SpyingRpcSchedulerFactory.class.getName());
|
||||
|
||||
// restart so that new config takes place
|
||||
afterClass();
|
||||
beforeClass();
|
||||
|
||||
TableName tableName = TableName.valueOf("foo");
|
||||
try (Admin admin = connection.getAdmin();
|
||||
RegionLocator rl = connection.getRegionLocator(tableName)) {
|
||||
|
||||
// create a table and prepare for a manual split
|
||||
UTIL.createTable(tableName, "cf1");
|
||||
|
||||
HRegionLocation loc = rl.getAllRegionLocations().get(0);
|
||||
HRegionInfo parent = loc.getRegionInfo();
|
||||
long rid = 1000;
|
||||
byte[] splitKey = Bytes.toBytes("a");
|
||||
HRegionInfo splitA = new HRegionInfo(parent.getTable(), parent.getStartKey(),
|
||||
splitKey, false, rid);
|
||||
HRegionInfo splitB = new HRegionInfo(parent.getTable(), splitKey,
|
||||
parent.getEndKey(), false, rid);
|
||||
|
||||
// find the meta server
|
||||
MiniHBaseCluster cluster = UTIL.getMiniHBaseCluster();
|
||||
int rsIndex = cluster.getServerWithMeta();
|
||||
HRegionServer rs;
|
||||
if (rsIndex >= 0) {
|
||||
rs = cluster.getRegionServer(rsIndex);
|
||||
} else {
|
||||
// it is in master
|
||||
rs = cluster.getMaster();
|
||||
}
|
||||
SpyingRpcScheduler scheduler = (SpyingRpcScheduler) rs.getRpcServer().getScheduler();
|
||||
long prevCalls = scheduler.numPriorityCalls;
|
||||
MetaTableAccessor.splitRegion(connection, parent, splitA, splitB, loc.getServerName(), 1);
|
||||
|
||||
assertTrue(prevCalls < scheduler.numPriorityCalls);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -31,6 +31,8 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
|
||||
import org.apache.hadoop.hbase.master.RegionState;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
|
||||
|
@ -249,6 +251,10 @@ public class TestMetaTableLocator {
|
|||
(GetRegionInfoRequest)Mockito.any())).thenThrow(connectException);
|
||||
Mockito.when(connection.getAdmin(Mockito.any(ServerName.class))).
|
||||
thenReturn(implementation);
|
||||
RpcControllerFactory controllerFactory = Mockito.mock(RpcControllerFactory.class);
|
||||
Mockito.when(controllerFactory.newController()).thenReturn(
|
||||
Mockito.mock(PayloadCarryingRpcController.class));
|
||||
Mockito.when(connection.getRpcControllerFactory()).thenReturn(controllerFactory);
|
||||
|
||||
ServerName sn = ServerName.valueOf("example.com", 1234, System.currentTimeMillis());
|
||||
MetaTableLocator.setMetaLocation(this.watcher,
|
||||
|
|
|
@ -63,6 +63,11 @@ public class HConnectionTestingUtility {
|
|||
if (connection == null) {
|
||||
connection = Mockito.mock(HConnectionImplementation.class);
|
||||
Mockito.when(connection.getConfiguration()).thenReturn(conf);
|
||||
Mockito.when(connection.getRpcControllerFactory()).thenReturn(
|
||||
Mockito.mock(RpcControllerFactory.class));
|
||||
// we need a real retrying caller
|
||||
RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(conf);
|
||||
Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
|
||||
ConnectionManager.CONNECTION_INSTANCES.put(connectionKey, connection);
|
||||
}
|
||||
return connection;
|
||||
|
|
|
@ -1256,8 +1256,8 @@ public class TestAdmin1 {
|
|||
try {
|
||||
AdminService.BlockingInterface admin = TEST_UTIL.getHBaseAdmin().getConnection()
|
||||
.getAdmin(regions.get(1).getSecond());
|
||||
ProtobufUtil.mergeRegions(admin, regions.get(1).getFirst(), regions.get(2).getFirst(), true,
|
||||
null);
|
||||
ProtobufUtil.mergeRegions(null, admin, regions.get(1).getFirst(), regions.get(2).getFirst(),
|
||||
true, null);
|
||||
} catch (MergeRegionException mm) {
|
||||
gotException = true;
|
||||
}
|
||||
|
|
|
@ -696,7 +696,7 @@ public class TestFromClientSide {
|
|||
public void testMaxKeyValueSize() throws Exception {
|
||||
byte [] TABLE = Bytes.toBytes("testMaxKeyValueSize");
|
||||
Configuration conf = TEST_UTIL.getConfiguration();
|
||||
String oldMaxSize = conf.get(TableConfiguration.MAX_KEYVALUE_SIZE_KEY);
|
||||
String oldMaxSize = conf.get(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY);
|
||||
Table ht = TEST_UTIL.createTable(TABLE, FAMILY);
|
||||
byte[] value = new byte[4 * 1024 * 1024];
|
||||
Put put = new Put(ROW);
|
||||
|
@ -704,7 +704,7 @@ public class TestFromClientSide {
|
|||
ht.put(put);
|
||||
try {
|
||||
TEST_UTIL.getConfiguration().setInt(
|
||||
TableConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024);
|
||||
ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, 2 * 1024 * 1024);
|
||||
// Create new table so we pick up the change in Configuration.
|
||||
try (Connection connection =
|
||||
ConnectionFactory.createConnection(TEST_UTIL.getConfiguration())) {
|
||||
|
@ -716,7 +716,7 @@ public class TestFromClientSide {
|
|||
}
|
||||
fail("Inserting a too large KeyValue worked, should throw exception");
|
||||
} catch(Exception e) {}
|
||||
conf.set(TableConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize);
|
||||
conf.set(ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, oldMaxSize);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -36,6 +36,8 @@ import org.apache.hadoop.hbase.MasterNotRunningException;
|
|||
import org.apache.hadoop.hbase.PleaseHoldException;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
|
||||
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
|
||||
|
@ -309,6 +311,14 @@ public class TestHBaseAdminNoCluster {
|
|||
}
|
||||
});
|
||||
Mockito.when(connection.getKeepAliveMasterService()).thenReturn(masterAdmin);
|
||||
RpcControllerFactory rpcControllerFactory = Mockito.mock(RpcControllerFactory.class);
|
||||
Mockito.when(connection.getRpcControllerFactory()).thenReturn(rpcControllerFactory);
|
||||
Mockito.when(rpcControllerFactory.newController()).thenReturn(
|
||||
Mockito.mock(PayloadCarryingRpcController.class));
|
||||
|
||||
// we need a real retrying caller
|
||||
RpcRetryingCallerFactory callerFactory = new RpcRetryingCallerFactory(configuration);
|
||||
Mockito.when(connection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
|
||||
|
||||
Admin admin = null;
|
||||
try {
|
||||
|
|
|
@ -607,7 +607,7 @@ public class TestScannersFromClientSide {
|
|||
byte[] regionName = hri.getRegionName();
|
||||
int i = cluster.getServerWith(regionName);
|
||||
HRegionServer rs = cluster.getRegionServer(i);
|
||||
ProtobufUtil.closeRegion(
|
||||
ProtobufUtil.closeRegion(null,
|
||||
rs.getRSRpcServices(), rs.getServerName(), regionName, false);
|
||||
long startTime = EnvironmentEdgeManager.currentTime();
|
||||
long timeOut = 300000;
|
||||
|
@ -630,7 +630,7 @@ public class TestScannersFromClientSide {
|
|||
if (ConfigUtil.useZKForAssignment(TEST_UTIL.getConfiguration())) {
|
||||
ZKAssign.createNodeOffline(zkw, hri, loc.getServerName());
|
||||
}
|
||||
ProtobufUtil.openRegion(rs.getRSRpcServices(), rs.getServerName(), hri);
|
||||
ProtobufUtil.openRegion(null, rs.getRSRpcServices(), rs.getServerName(), hri);
|
||||
startTime = EnvironmentEdgeManager.currentTime();
|
||||
while (true) {
|
||||
if (rs.getOnlineRegion(regionName) != null) {
|
||||
|
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* 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.ipc;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class DelegatingRpcScheduler extends RpcScheduler {
|
||||
protected RpcScheduler delegate;
|
||||
|
||||
public DelegatingRpcScheduler(RpcScheduler delegate) {
|
||||
this.delegate = delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
delegate.stop();
|
||||
}
|
||||
@Override
|
||||
public void start() {
|
||||
delegate.start();
|
||||
}
|
||||
@Override
|
||||
public void init(Context context) {
|
||||
delegate.init(context);
|
||||
}
|
||||
@Override
|
||||
public int getReplicationQueueLength() {
|
||||
return delegate.getReplicationQueueLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPriorityQueueLength() {
|
||||
return delegate.getPriorityQueueLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getGeneralQueueLength() {
|
||||
return delegate.getGeneralQueueLength();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getActiveRpcHandlerCount() {
|
||||
return delegate.getActiveRpcHandlerCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean dispatch(CallRunner task) throws IOException, InterruptedException {
|
||||
return delegate.dispatch(task);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumGeneralCallsDropped() {
|
||||
return delegate.getNumGeneralCallsDropped();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getNumLifoModeSwitches() {
|
||||
return delegate.getNumLifoModeSwitches();
|
||||
}
|
||||
}
|
|
@ -193,7 +193,7 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
ProtobufUtil.getOnlineRegions(hrs.getRSRpcServices())) {
|
||||
if (hri.getTable().equals(table)) {
|
||||
// splitRegion doesn't work if startkey/endkey are null
|
||||
ProtobufUtil.split(hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2)); // hard code split
|
||||
ProtobufUtil.split(null, hrs.getRSRpcServices(), hri, rowkey(ROWCOUNT / 2));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -469,9 +469,10 @@ public class TestLoadIncrementalHFilesSplitRecovery {
|
|||
final AtomicInteger countedLqis = new AtomicInteger();
|
||||
LoadIncrementalHFiles loader = new LoadIncrementalHFiles(util.getConfiguration()) {
|
||||
|
||||
@Override
|
||||
protected List<LoadQueueItem> groupOrSplit(
|
||||
Multimap<ByteBuffer, LoadQueueItem> regionGroups,
|
||||
final LoadQueueItem item, final HTable htable,
|
||||
final LoadQueueItem item, final Table htable,
|
||||
final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
|
||||
List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
|
||||
if (lqis != null) {
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
package org.apache.hadoop.hbase.master;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.net.InetAddress;
|
||||
|
||||
|
@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
|
|||
import org.apache.hadoop.hbase.Server;
|
||||
import org.apache.hadoop.hbase.ServerName;
|
||||
import org.apache.hadoop.hbase.client.ClusterConnection;
|
||||
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
|
||||
import org.apache.hadoop.hbase.testclassification.SmallTests;
|
||||
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
|
||||
|
|
|
@ -349,7 +349,7 @@ public class TestMasterFailover {
|
|||
region = enabledRegions.remove(0);
|
||||
regionsThatShouldBeOnline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, serverName);
|
||||
ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
|
||||
ProtobufUtil.openRegion(null, hrs.getRSRpcServices(), hrs.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
|
@ -364,7 +364,7 @@ public class TestMasterFailover {
|
|||
region = disabledRegions.remove(0);
|
||||
regionsThatShouldBeOffline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, serverName);
|
||||
ProtobufUtil.openRegion(hrs.getRSRpcServices(), hrs.getServerName(), region);
|
||||
ProtobufUtil.openRegion(null, hrs.getRSRpcServices(), hrs.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
RegionTransition rt = RegionTransition.parseFrom(bytes);
|
||||
|
@ -752,7 +752,7 @@ public class TestMasterFailover {
|
|||
region = enabledRegions.remove(0);
|
||||
regionsThatShouldBeOnline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
|
||||
ProtobufUtil.openRegion(null, hrsDead.getRSRpcServices(),
|
||||
hrsDead.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
|
@ -768,7 +768,7 @@ public class TestMasterFailover {
|
|||
region = disabledRegions.remove(0);
|
||||
regionsThatShouldBeOffline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
|
||||
ProtobufUtil.openRegion(null, hrsDead.getRSRpcServices(),
|
||||
hrsDead.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
|
@ -788,7 +788,7 @@ public class TestMasterFailover {
|
|||
region = enabledRegions.remove(0);
|
||||
regionsThatShouldBeOnline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
|
||||
ProtobufUtil.openRegion(null, hrsDead.getRSRpcServices(),
|
||||
hrsDead.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
|
@ -807,7 +807,7 @@ public class TestMasterFailover {
|
|||
region = disabledRegions.remove(0);
|
||||
regionsThatShouldBeOffline.add(region);
|
||||
ZKAssign.createNodeOffline(zkw, region, deadServerName);
|
||||
ProtobufUtil.openRegion(hrsDead.getRSRpcServices(),
|
||||
ProtobufUtil.openRegion(null, hrsDead.getRSRpcServices(),
|
||||
hrsDead.getServerName(), region);
|
||||
while (true) {
|
||||
byte [] bytes = ZKAssign.getData(zkw, region.getEncodedName());
|
||||
|
|
|
@ -217,7 +217,7 @@ public class TestZKBasedOpenCloseRegion {
|
|||
Whitebox.setInternalState(regionServer, "tableDescriptors", htd);
|
||||
Mockito.doThrow(new IOException()).when(htd).get((TableName) Mockito.any());
|
||||
try {
|
||||
ProtobufUtil.openRegion(regionServer.getRSRpcServices(),
|
||||
ProtobufUtil.openRegion(null, regionServer.getRSRpcServices(),
|
||||
regionServer.getServerName(), REGIONINFO);
|
||||
fail("It should throw IOException ");
|
||||
} catch (IOException e) {
|
||||
|
|
|
@ -372,7 +372,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, user, actions);
|
||||
ProtobufUtil.grant(null, protocol, user, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -395,7 +395,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, user, actions);
|
||||
ProtobufUtil.revoke(null, protocol, user, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -418,7 +418,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, user, namespace, actions);
|
||||
ProtobufUtil.grant(null, protocol, user, namespace, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -483,7 +483,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, user, namespace, actions);
|
||||
ProtobufUtil.revoke(null, protocol, user, namespace, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -507,7 +507,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, user, table, family, qualifier, actions);
|
||||
ProtobufUtil.grant(null, protocol, user, table, family, qualifier, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
@ -573,7 +573,7 @@ public class SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, user, table, family, qualifier, actions);
|
||||
ProtobufUtil.revoke(null, protocol, user, table, family, qualifier, actions);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
|
|
|
@ -1165,7 +1165,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
|
||||
ProtobufUtil.grant(null, protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
|
||||
Action.READ);
|
||||
}
|
||||
return null;
|
||||
|
@ -1180,7 +1180,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
|
||||
ProtobufUtil.revoke(null, protocol, USER_RO.getShortName(), TEST_TABLE, TEST_FAMILY, null,
|
||||
Action.READ);
|
||||
}
|
||||
return null;
|
||||
|
@ -1195,7 +1195,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.getUserPermissions(protocol, TEST_TABLE);
|
||||
ProtobufUtil.getUserPermissions(null, protocol, TEST_TABLE);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -1209,7 +1209,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.getUserPermissions(protocol);
|
||||
ProtobufUtil.getUserPermissions(null, protocol);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -1619,7 +1619,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol, tableName);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
@ -1646,7 +1646,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol, tableName);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
@ -1670,7 +1670,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol, tableName);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
@ -1690,7 +1690,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol, tableName);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
@ -1710,7 +1710,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(tableName.getName());
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol, tableName);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol, tableName);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
@ -1733,7 +1733,7 @@ public class TestAccessController extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
perms = ProtobufUtil.getUserPermissions(protocol);
|
||||
perms = ProtobufUtil.getUserPermissions(null, protocol);
|
||||
} finally {
|
||||
acl.close();
|
||||
}
|
||||
|
|
|
@ -231,6 +231,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
@Test
|
||||
public void testModifyNamespace() throws Exception {
|
||||
AccessTestAction modifyNamespace = new AccessTestAction() {
|
||||
@Override
|
||||
public Object run() throws Exception {
|
||||
ACCESS_CONTROLLER.preModifyNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
|
||||
NamespaceDescriptor.create(TEST_NAMESPACE).addConfiguration("abc", "156").build());
|
||||
|
@ -356,7 +357,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, testUser, TEST_NAMESPACE, Action.WRITE);
|
||||
ProtobufUtil.grant(null, protocol, testUser, TEST_NAMESPACE, Action.WRITE);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
@ -370,7 +371,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.grant(protocol, USER_GROUP_NS_ADMIN.getShortName(),
|
||||
ProtobufUtil.grant(null, protocol, USER_GROUP_NS_ADMIN.getShortName(),
|
||||
TEST_NAMESPACE, Action.READ);
|
||||
}
|
||||
return null;
|
||||
|
@ -378,6 +379,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
};
|
||||
|
||||
AccessTestAction revokeAction = new AccessTestAction() {
|
||||
@Override
|
||||
public Object run() throws Exception {
|
||||
try(Connection conn = ConnectionFactory.createConnection(conf);
|
||||
Table acl = conn.getTable(AccessControlLists.ACL_TABLE_NAME)) {
|
||||
|
@ -385,13 +387,14 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, testUser, TEST_NAMESPACE, Action.WRITE);
|
||||
ProtobufUtil.revoke(null, protocol, testUser, TEST_NAMESPACE, Action.WRITE);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
AccessTestAction revokeNamespaceAction = new AccessTestAction() {
|
||||
@Override
|
||||
public Object run() throws Exception {
|
||||
Connection connection = ConnectionFactory.createConnection(conf);
|
||||
Table acl = connection.getTable(AccessControlLists.ACL_TABLE_NAME);
|
||||
|
@ -400,7 +403,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.revoke(protocol, USER_GROUP_NS_ADMIN.getShortName(),
|
||||
ProtobufUtil.revoke(null, protocol, USER_GROUP_NS_ADMIN.getShortName(),
|
||||
TEST_NAMESPACE, Action.READ);
|
||||
} finally {
|
||||
acl.close();
|
||||
|
@ -418,7 +421,7 @@ public class TestNamespaceCommands extends SecureTestUtil {
|
|||
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
|
||||
AccessControlService.BlockingInterface protocol =
|
||||
AccessControlService.newBlockingStub(service);
|
||||
ProtobufUtil.getUserPermissions(protocol, Bytes.toBytes(TEST_NAMESPACE));
|
||||
ProtobufUtil.getUserPermissions(null, protocol, Bytes.toBytes(TEST_NAMESPACE));
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue