diff --git a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
index fb9c6a6e960..26111c2fd9d 100644
--- a/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
+++ b/hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java
@@ -123,9 +123,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
@Override
public void stop(CoprocessorEnvironment env) {
- if (accessChecker.getAuthManager() != null) {
- TableAuthManager.release(accessChecker.getAuthManager());
- }
+ accessChecker.stop();
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index 08a20339f76..0ccdc278245 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -192,6 +192,7 @@ import org.apache.hadoop.hbase.regionserver.RSRpcServices;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.security.access.AccessController;
+import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.visibility.VisibilityController;
import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
@@ -249,6 +250,24 @@ public class MasterRpcServices extends RSRpcServices
return new MasterAnnotationReadingPriorityFunction(this);
}
+ /**
+ * Checks for the following pre-checks in order:
+ *
+ *
Master is initialized
+ *
Rpc caller has admin permissions
+ *
+ * @param requestName name of rpc request. Used in reporting failures to provide context.
+ * @throws ServiceException If any of the above listed pre-check fails.
+ */
+ private void rpcPreCheck(String requestName) throws ServiceException {
+ try {
+ master.checkInitialized();
+ requirePermission(requestName, Permission.Action.ADMIN);
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+ }
+
enum BalanceSwitchMode {
SYNC,
ASYNC
@@ -292,25 +311,6 @@ public class MasterRpcServices extends RSRpcServices
return switchBalancer(b, BalanceSwitchMode.SYNC);
}
- /**
- * Sets normalizer on/off flag in ZK.
- */
- public boolean normalizerSwitch(boolean on) {
- boolean oldValue = master.getRegionNormalizerTracker().isNormalizerOn();
- boolean newValue = on;
- try {
- try {
- master.getRegionNormalizerTracker().setNormalizerOn(newValue);
- } catch (KeeperException ke) {
- throw new IOException(ke);
- }
- LOG.info(master.getClientIdAuditPrefix() + " set normalizerSwitch=" + newValue);
- } catch (IOException ioe) {
- LOG.warn("Error flipping normalizer switch", ioe);
- }
- return oldValue;
- }
-
/**
* @return list of blocking services and their security info classes that this server supports
*/
@@ -649,11 +649,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public EnableCatalogJanitorResponse enableCatalogJanitor(RpcController c,
EnableCatalogJanitorRequest req) throws ServiceException {
- try {
- master.checkInitialized();
- } catch (IOException ioe) {
- throw new ServiceException(ioe);
- }
+ rpcPreCheck("enableCatalogJanitor");
return EnableCatalogJanitorResponse.newBuilder().setPrevValue(
master.catalogJanitorChore.setEnabled(req.getEnable())).build();
}
@@ -661,11 +657,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public SetCleanerChoreRunningResponse setCleanerChoreRunning(RpcController c,
SetCleanerChoreRunningRequest req) throws ServiceException {
- try {
- master.checkInitialized();
- } catch (IOException ioe) {
- throw new ServiceException(ioe);
- }
+ rpcPreCheck("setCleanerChoreRunning");
boolean prevValue =
master.getLogCleaner().getEnabled() && master.getHFileCleaner().getEnabled();
master.getLogCleaner().setEnabled(req.getOn());
@@ -690,10 +682,9 @@ public class MasterRpcServices extends RSRpcServices
@Override
public ClientProtos.CoprocessorServiceResponse execMasterService(final RpcController controller,
final ClientProtos.CoprocessorServiceRequest request) throws ServiceException {
+ rpcPreCheck("execMasterService");
try {
- master.checkInitialized();
ServerRpcController execController = new ServerRpcController();
-
ClientProtos.CoprocessorServiceCall call = request.getCall();
String serviceName = call.getServiceName();
String methodName = call.getMethodName();
@@ -749,8 +740,8 @@ public class MasterRpcServices extends RSRpcServices
@Override
public ExecProcedureResponse execProcedure(RpcController controller,
ExecProcedureRequest request) throws ServiceException {
+ rpcPreCheck("execProcedure");
try {
- master.checkInitialized();
ProcedureDescription desc = request.getProcedure();
MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
desc.getSignature());
@@ -758,12 +749,8 @@ public class MasterRpcServices extends RSRpcServices
throw new ServiceException("The procedure is not registered: "
+ desc.getSignature());
}
-
- LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
- + desc.getSignature());
-
+ LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
mpm.execProcedure(desc);
-
// send back the max amount of time the client should wait for the procedure
// to complete
long waitTime = SnapshotDescriptionUtils.DEFAULT_MAX_WAIT_TIME;
@@ -784,21 +771,16 @@ public class MasterRpcServices extends RSRpcServices
@Override
public ExecProcedureResponse execProcedureWithRet(RpcController controller,
ExecProcedureRequest request) throws ServiceException {
+ rpcPreCheck("execProcedureWithRet");
try {
- master.checkInitialized();
ProcedureDescription desc = request.getProcedure();
- MasterProcedureManager mpm = master.getMasterProcedureManagerHost().getProcedureManager(
- desc.getSignature());
+ MasterProcedureManager mpm =
+ master.getMasterProcedureManagerHost().getProcedureManager(desc.getSignature());
if (mpm == null) {
- throw new ServiceException("The procedure is not registered: "
- + desc.getSignature());
+ throw new ServiceException("The procedure is not registered: " + desc.getSignature());
}
-
- LOG.info(master.getClientIdAuditPrefix() + " procedure request for: "
- + desc.getSignature());
-
+ LOG.info(master.getClientIdAuditPrefix() + " procedure request for: " + desc.getSignature());
byte[] data = mpm.execProcedureWithRet(desc);
-
ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
// set return data if available
if (data != null) {
@@ -1110,8 +1092,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public AbortProcedureResponse abortProcedure(
- RpcController rpcController,
- AbortProcedureRequest request) throws ServiceException {
+ RpcController rpcController, AbortProcedureRequest request) throws ServiceException {
try {
AbortProcedureResponse.Builder response = AbortProcedureResponse.newBuilder();
boolean abortResult =
@@ -1362,8 +1343,8 @@ public class MasterRpcServices extends RSRpcServices
@Override
public RunCatalogScanResponse runCatalogScan(RpcController c,
RunCatalogScanRequest req) throws ServiceException {
+ rpcPreCheck("runCatalogScan");
try {
- master.checkInitialized();
return ResponseConverter.buildRunCatalogScanResponse(master.catalogJanitorChore.scan());
} catch (IOException ioe) {
throw new ServiceException(ioe);
@@ -1373,14 +1354,9 @@ public class MasterRpcServices extends RSRpcServices
@Override
public RunCleanerChoreResponse runCleanerChore(RpcController c, RunCleanerChoreRequest req)
throws ServiceException {
- try {
- master.checkInitialized();
- Boolean result = master.getHFileCleaner().runCleaner()
- && master.getLogCleaner().runCleaner();
- return ResponseConverter.buildRunCleanerChoreResponse(result);
- } catch (IOException ioe) {
- throw new ServiceException(ioe);
- }
+ rpcPreCheck("runCleanerChore");
+ Boolean result = master.getHFileCleaner().runCleaner() && master.getLogCleaner().runCleaner();
+ return ResponseConverter.buildRunCleanerChoreResponse(result);
}
@Override
@@ -1614,6 +1590,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public NormalizeResponse normalize(RpcController controller,
NormalizeRequest request) throws ServiceException {
+ rpcPreCheck("normalize");
try {
return NormalizeResponse.newBuilder().setNormalizerRan(master.normalizeRegions()).build();
} catch (IOException | CoordinatedStateException ex) {
@@ -1624,13 +1601,18 @@ public class MasterRpcServices extends RSRpcServices
@Override
public SetNormalizerRunningResponse setNormalizerRunning(RpcController controller,
SetNormalizerRunningRequest request) throws ServiceException {
+ rpcPreCheck("setNormalizerRunning");
+
+ // Sets normalizer on/off flag in ZK.
+ boolean prevValue = master.getRegionNormalizerTracker().isNormalizerOn();
+ boolean newValue = request.getOn();
try {
- master.checkInitialized();
- boolean prevValue = normalizerSwitch(request.getOn());
- return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
- } catch (IOException ioe) {
- throw new ServiceException(ioe);
+ master.getRegionNormalizerTracker().setNormalizerOn(newValue);
+ } catch (KeeperException ke) {
+ LOG.warn("Error flipping normalizer switch", ke);
}
+ LOG.info(master.getClientIdAuditPrefix() + " set normalizerSwitch=" + newValue);
+ return SetNormalizerRunningResponse.newBuilder().setPrevNormalizerValue(prevValue).build();
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 37ea94b8a60..8512c2f2985 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -625,7 +625,7 @@ public class HRegionServer extends HasThread implements
}
this.configurationManager = new ConfigurationManager();
- rpcServices.start();
+ rpcServices.start(zooKeeper);
putUpWebUI();
this.walRoller = new LogRoller(this, this);
this.choreService = new ChoreService(getServerName().toString(), true);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
index cf3922b8286..9da4cbb2836 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RSRpcServices.java
@@ -187,6 +187,8 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenPriorityRegionHandler;
import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessChecker;
+import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.DNS;
@@ -198,6 +200,7 @@ import org.apache.hadoop.hbase.wal.WAL;
import org.apache.hadoop.hbase.wal.WALKey;
import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
/**
@@ -286,6 +289,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
*/
private final int rowSizeWarnThreshold;
+ // We want to vet all accesses at the point of entry itself; limiting scope of access checker
+ // instance to only this class to prevent its use from spreading deeper into implementation.
+ // Initialized in start() since AccessChecker needs ZKWatcher which is created by HRegionServer
+ // after RSRpcServices constructor and before start() is called.
+ // Initialized only if authorization is enabled, else remains null.
+ private AccessChecker accessChecker;
+
/**
* Holder class which holds the RegionScanner, nextCallSeq and RpcCallbacks together.
*/
@@ -372,6 +382,24 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return builder.setIndex(index).build();
}
+ /**
+ * Checks for the following pre-checks in order:
+ *
+ *
RegionServer is running
+ *
If authorization is enabled, then RPC caller has ADMIN permissions
+ *
+ * @param requestName name of rpc request. Used in reporting failures to provide context.
+ * @throws ServiceException If any of the above listed pre-check fails.
+ */
+ private void rpcPreCheck(String requestName) throws ServiceException {
+ try {
+ checkOpen();
+ requirePermission(requestName, Permission.Action.ADMIN);
+ } catch (IOException ioe) {
+ throw new ServiceException(ioe);
+ }
+ }
+
/**
* Starts the nonce operation for a mutation, if needed.
* @param mutation Mutation.
@@ -1166,6 +1194,13 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return new AnnotationReadingPriorityFunction(this);
}
+ protected void requirePermission(String request, Permission.Action perm) throws IOException {
+ if (accessChecker != null) {
+ accessChecker.requirePermission(RpcServer.getRequestUser(), request, perm);
+ }
+ }
+
+
public static String getHostname(Configuration conf, boolean isMaster)
throws UnknownHostException {
String hostname = conf.get(isMaster? HRegionServer.MASTER_HOSTNAME_KEY :
@@ -1289,21 +1324,26 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
return regionServer.getRegionServerQuotaManager();
}
- void start() {
+ void start(ZooKeeperWatcher zkWatcher) {
+ if (AccessChecker.isAuthorizationSupported(getConfiguration())) {
+ accessChecker = new AccessChecker(getConfiguration(), zkWatcher);
+ }
this.scannerIdGenerator = new ScannerIdGenerator(this.regionServer.serverName);
rpcServer.start();
}
void stop() {
+ if (accessChecker != null) {
+ accessChecker.stop();
+ }
closeAllScanners();
rpcServer.stop();
}
/**
* Called to verify that this server is up and running.
- *
- * @throws IOException
*/
+ // TODO : Rename this and HMaster#checkInitialized to isRunning() (or a better name).
protected void checkOpen() throws IOException {
if (regionServer.isAborted()) {
throw new RegionServerAbortedException("Server " + regionServer.serverName + " aborting");
@@ -3192,6 +3232,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
@Override
public CoprocessorServiceResponse execRegionServerService(RpcController controller,
CoprocessorServiceRequest request) throws ServiceException {
+ rpcPreCheck("execRegionServerService");
return regionServer.execRegionServerService(controller, request);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
index 6afc96d868f..da88211699d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessChecker.java
@@ -38,6 +38,9 @@ import org.slf4j.LoggerFactory;
public final class AccessChecker {
private static final Logger AUDITLOG =
LoggerFactory.getLogger("SecurityLogger." + AccessChecker.class.getName());
+ // TODO: we should move to a design where we don't even instantiate an AccessChecker if
+ // authorization is not enabled (like in RSRpcServices), instead of always instantiating one and
+ // calling requireXXX() only to do nothing (since authorizationEnabled will be false).
private TableAuthManager authManager;
/**
* if we are active, usually false, only true if "hbase.security.authorization"
@@ -57,8 +60,6 @@ public final class AccessChecker {
*/
public AccessChecker(final Configuration conf, final ZooKeeperWatcher zkw)
throws RuntimeException {
- // If zk is null or IOException while obtaining auth manager,
- // throw RuntimeException so that the coprocessor is unloaded.
if (zkw != null) {
try {
this.authManager = TableAuthManager.getOrCreate(zkw, conf);
@@ -71,6 +72,13 @@ public final class AccessChecker {
authorizationEnabled = isAuthorizationSupported(conf);
}
+ /**
+ * Releases {@link TableAuthManager}'s reference.
+ */
+ public void stop() {
+ TableAuthManager.release(authManager);
+ }
+
public TableAuthManager getAuthManager() {
return authManager;
}
@@ -159,6 +167,9 @@ public final class AccessChecker {
*/
public void requireAccess(User user, String request, TableName tableName,
Action... permissions) throws IOException {
+ if (!authorizationEnabled) {
+ return;
+ }
AuthResult result = null;
for (Action permission : permissions) {
@@ -173,7 +184,7 @@ public final class AccessChecker {
}
}
logResult(result);
- if (authorizationEnabled && !result.isAllowed()) {
+ if (!result.isAllowed()) {
throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
}
}
@@ -198,7 +209,10 @@ public final class AccessChecker {
*/
public void requireGlobalPermission(User user, String request, Action perm, TableName tableName,
Map> familyMap) throws IOException {
- AuthResult result = null;
+ if (!authorizationEnabled) {
+ return;
+ }
+ AuthResult result;
if (authManager.authorize(user, perm)) {
result = AuthResult.allow(request, "Global check allowed", user, perm, tableName, familyMap);
result.getParams().setTableName(tableName).setFamilies(familyMap);
@@ -207,11 +221,9 @@ public final class AccessChecker {
result = AuthResult.deny(request, "Global check failed", user, perm, tableName, familyMap);
result.getParams().setTableName(tableName).setFamilies(familyMap);
logResult(result);
- if (authorizationEnabled) {
- throw new AccessDeniedException("Insufficient permissions for user '" +
- (user != null ? user.getShortName() : "null") +"' (global, action=" +
- perm.toString() + ")");
- }
+ throw new AccessDeniedException(
+ "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+ + "' (global, action=" + perm.toString() + ")");
}
}
@@ -224,7 +236,10 @@ public final class AccessChecker {
*/
public void requireGlobalPermission(User user, String request, Action perm,
String namespace) throws IOException {
- AuthResult authResult = null;
+ if (!authorizationEnabled) {
+ return;
+ }
+ AuthResult authResult;
if (authManager.authorize(user, perm)) {
authResult = AuthResult.allow(request, "Global check allowed", user, perm, null);
authResult.getParams().setNamespace(namespace);
@@ -233,11 +248,9 @@ public final class AccessChecker {
authResult = AuthResult.deny(request, "Global check failed", user, perm, null);
authResult.getParams().setNamespace(namespace);
logResult(authResult);
- if (authorizationEnabled) {
- throw new AccessDeniedException("Insufficient permissions for user '" +
- (user != null ? user.getShortName() : "null") +"' (global, action=" +
- perm.toString() + ")");
- }
+ throw new AccessDeniedException(
+ "Insufficient permissions for user '" + (user != null ? user.getShortName() : "null")
+ + "' (global, action=" + perm.toString() + ")");
}
}
@@ -248,23 +261,24 @@ public final class AccessChecker {
*/
public void requireNamespacePermission(User user, String request, String namespace,
Action... permissions) throws IOException {
+ if (!authorizationEnabled) {
+ return;
+ }
AuthResult result = null;
for (Action permission : permissions) {
if (authManager.authorize(user, namespace, permission)) {
- result = AuthResult.allow(request, "Namespace permission granted",
- user, permission, namespace);
+ result =
+ AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
break;
} else {
// rest of the world
- result = AuthResult.deny(request, "Insufficient permissions", user,
- permission, namespace);
+ result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
}
}
logResult(result);
- if (authorizationEnabled && !result.isAllowed()) {
- throw new AccessDeniedException("Insufficient permissions "
- + result.toContextString());
+ if (!result.isAllowed()) {
+ throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
}
}
@@ -273,28 +287,29 @@ public final class AccessChecker {
* @param namespace The given namespace
* @param permissions Actions being requested
*/
- public void requireNamespacePermission(User user, String request, String namespace, TableName tableName,
- Map> familyMap, Action... permissions)
- throws IOException {
+ public void requireNamespacePermission(User user, String request, String namespace,
+ TableName tableName, Map> familyMap,
+ Action... permissions) throws IOException {
+ if (!authorizationEnabled) {
+ return;
+ }
AuthResult result = null;
for (Action permission : permissions) {
if (authManager.authorize(user, namespace, permission)) {
- result = AuthResult.allow(request, "Namespace permission granted",
- user, permission, namespace);
+ result =
+ AuthResult.allow(request, "Namespace permission granted", user, permission, namespace);
result.getParams().setTableName(tableName).setFamilies(familyMap);
break;
} else {
// rest of the world
- result = AuthResult.deny(request, "Insufficient permissions", user,
- permission, namespace);
+ result = AuthResult.deny(request, "Insufficient permissions", user, permission, namespace);
result.getParams().setTableName(tableName).setFamilies(familyMap);
}
}
logResult(result);
- if (authorizationEnabled && !result.isAllowed()) {
- throw new AccessDeniedException("Insufficient permissions "
- + result.toContextString());
+ if (!result.isAllowed()) {
+ throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
}
}
}
\ No newline at end of file
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index fd0a704c270..87b79477ad0 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -846,15 +846,14 @@ public class AccessController extends BaseMasterAndRegionObserver
// set the user-provider.
this.userProvider = UserProvider.instantiate(env.getConfiguration());
+ // Throws RuntimeException if fails to load TableAuthManager so that coprocessor is unloaded.
accessChecker = new AccessChecker(env.getConfiguration(), zk);
tableAcls = new MapMaker().weakValues().makeMap();
}
@Override
public void stop(CoprocessorEnvironment env) {
- if (getAuthManager()!= null) {
- TableAuthManager.release(getAuthManager());
- }
+ accessChecker.stop();
}
@Override
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index c1d6e8faa9e..3d61bc699d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -217,15 +217,16 @@ public class TestAccessController extends SecureTestUtil {
conf.setBoolean(AccessControlConstants.EXEC_PERMISSION_CHECKS_KEY, true);
TEST_UTIL.startMiniCluster();
- MasterCoprocessorHost cpHost =
+ MasterCoprocessorHost masterCpHost =
TEST_UTIL.getMiniHBaseCluster().getMaster().getMasterCoprocessorHost();
- cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
- ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
- CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+ masterCpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
+ ACCESS_CONTROLLER = (AccessController) masterCpHost.findCoprocessor(
+ AccessController.class.getName());
+ CP_ENV = masterCpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
Coprocessor.PRIORITY_HIGHEST, 1, conf);
- RegionServerCoprocessorHost rsHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
+ RegionServerCoprocessorHost rsCpHost = TEST_UTIL.getMiniHBaseCluster().getRegionServer(0)
.getRegionServerCoprocessorHost();
- RSCP_ENV = rsHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
+ RSCP_ENV = rsCpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
Coprocessor.PRIORITY_HIGHEST, 1, conf);
// Wait for the ACL table to become available
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
new file mode 100644
index 00000000000..ea0af354ba9
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAdminOnlyOperations.java
@@ -0,0 +1,311 @@
+
+/**
+ * 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.security.access;
+
+import static org.apache.hadoop.hbase.AuthUtil.toGroupEntry;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+import com.google.protobuf.Service;
+import com.google.protobuf.ServiceException;
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.HashMap;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.SingletonCoprocessorService;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestProtos;
+import org.apache.hadoop.hbase.ipc.protobuf.generated.TestRpcServiceProtos;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+/**
+ * This class tests operations in MasterRpcServices which require ADMIN access.
+ * It doesn't test all operations which require ADMIN access, only those which get vetted within
+ * MasterRpcServices at the point of entry itself (unlike old approach of using
+ * hooks in AccessController).
+ *
+ * Sidenote:
+ * There is one big difference between how security tests for AccessController hooks work, and how
+ * the tests in this class for security in MasterRpcServices work.
+ * The difference arises because of the way AC & MasterRpcServices get the user.
+ *
+ * In AccessController, it first checks if there is an active rpc user in ObserverContext. If not,
+ * it uses UserProvider for current user. This *might* make sense in the context of coprocessors,
+ * because they can be called outside the context of RPCs.
+ * But in the context of MasterRpcServices, only one way makes sense - RPCServer.getRequestUser().
+ *
+ * In AC tests, when we do FooUser.runAs on AccessController instance directly, it bypasses
+ * the rpc framework completely, but works because UserProvider provides the correct user, i.e.
+ * FooUser in this case.
+ *
+ * But this doesn't work for the tests here, so we go around by doing complete RPCs.
+ */
+@Category({SecurityTests.class, MediumTests.class})
+public class TestAdminOnlyOperations {
+
+ private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+ private static Configuration conf;
+
+ // user granted with all global permission
+ private static User USER_ADMIN;
+ // user without admin permissions
+ private static User USER_NON_ADMIN;
+
+ private static final String GROUP_ADMIN = "admin_group";
+ private static User USER_GROUP_ADMIN;
+
+ // Dummy service to test execService calls. Needs to be public so can be loaded as Coprocessor.
+ public static class DummyCpService implements Coprocessor, CoprocessorService,
+ SingletonCoprocessorService {
+ public DummyCpService() {}
+ public void start(CoprocessorEnvironment env) {}
+ public void stop(CoprocessorEnvironment env) {}
+
+ @Override
+ public Service getService() {
+ return mock(TestRpcServiceProtos.TestProtobufRpcProto.class);
+ }
+ }
+
+ private static void enableSecurity(Configuration conf) throws IOException {
+ conf.set("hadoop.security.authorization", "false");
+ conf.set("hadoop.security.authentication", "simple");
+ conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+ "," + DummyCpService.class.getName());
+ conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, AccessController.class.getName());
+ conf.set(CoprocessorHost.REGIONSERVER_COPROCESSOR_CONF_KEY, AccessController.class.getName() +
+ "," + DummyCpService.class.getName());
+ conf.set(User.HBASE_SECURITY_AUTHORIZATION_CONF_KEY, "true");
+ SecureTestUtil.configureSuperuser(conf);
+ }
+
+ @BeforeClass
+ public static void setup() throws Exception {
+ conf = TEST_UTIL.getConfiguration();
+
+ // Enable security
+ enableSecurity(conf);
+ TEST_UTIL.startMiniCluster();
+
+ // Wait for the ACL table to become available
+ TEST_UTIL.waitUntilAllRegionsAssigned(AccessControlLists.ACL_TABLE_NAME);
+
+ // Create users
+ USER_ADMIN = User.createUserForTesting(conf, "admin", new String[0]);
+ USER_NON_ADMIN = User.createUserForTesting(conf, "non_admin", new String[0]);
+ USER_GROUP_ADMIN =
+ User.createUserForTesting(conf, "user_group_admin", new String[] { GROUP_ADMIN });
+
+ // Assign permissions to users and groups
+ SecureTestUtil.grantGlobal(TEST_UTIL, USER_ADMIN.getShortName(), Permission.Action.ADMIN);
+ SecureTestUtil.grantGlobal(TEST_UTIL, toGroupEntry(GROUP_ADMIN), Permission.Action.ADMIN);
+ // No permissions to USER_NON_ADMIN
+ }
+
+ interface Action {
+ void run(Admin admin) throws Exception;
+ }
+
+ private void verifyAllowed(User user, final Action action) throws Exception {
+ user.runAs(new PrivilegedExceptionAction