Author: apurtell
Date: Mon Apr 1 16:24:53 2013
New Revision: 1463189
URL: http://svn.apache.org/r1463189
Log:
HBASE-8213. Global authorization may lose efficacy
Modified:
hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java?rev=1463189&r1=1463188&r2=1463189&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
(original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/TableAuthManager.java
Mon Apr 1 16:24:53 2013
@@ -101,15 +101,16 @@ public class TableAuthManager {
private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
throws IOException {
this.conf = conf;
+
+ // initialize global permissions based on configuration
+ globalCache = initGlobal(conf);
+
this.zkperms = new ZKPermissionWatcher(watcher, this, conf);
try {
this.zkperms.start();
} catch (KeeperException ke) {
LOG.error("ZooKeeper initialization failed", ke);
}
-
- // initialize global permissions based on configuration
- globalCache = initGlobal(conf);
}
/**
Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java?rev=1463189&r1=1463188&r2=1463189&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
(original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
Mon Apr 1 16:24:53 2013
@@ -29,7 +29,10 @@ import java.security.PrivilegedActionExc
import java.security.PrivilegedExceptionAction;
import java.util.List;
import java.util.Map;
+import java.util.NavigableMap;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -43,6 +46,7 @@ import org.apache.hadoop.hbase.HRegionIn
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.LargeTests;
+import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete;
@@ -69,13 +73,19 @@ import org.apache.hadoop.hbase.protobuf.
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
import org.apache.hadoop.hbase.regionserver.ScanType;
import org.apache.hadoop.hbase.exceptions.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.access.AccessControlLists;
+import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.security.access.UserPermission;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.JVMClusterUtil;
+
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Before;
@@ -94,6 +104,7 @@ import com.google.protobuf.ServiceExcept
@Category(LargeTests.class)
@SuppressWarnings("rawtypes")
public class TestAccessController {
+ private static final Log LOG = LogFactory.getLog(TestAccessController.class);
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static Configuration conf;
@@ -113,6 +124,7 @@ public class TestAccessController {
private static User USER_NONE;
private static byte[] TEST_TABLE = Bytes.toBytes("testtable");
+ private static byte[] TEST_TABLE2 = Bytes.toBytes("testtable2");
private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
private static MasterCoprocessorEnvironment CP_ENV;
@@ -1887,4 +1899,83 @@ public class TestAccessController {
verifyAllowed(deleteAction, SUPERUSER, USER_ADMIN);
verifyDenied(cloneAction, USER_CREATE, USER_RW, USER_RO, USER_NONE, USER_OWNER);
}
+
+ @Test
+ public void testGlobalAuthorizationForNewRegisteredRS() throws Exception {
+ LOG.debug("Test for global authorization for a new registered RegionServer.");
+ MiniHBaseCluster hbaseCluster = TEST_UTIL.getHBaseCluster();
+
+ // Since each RegionServer running on different user, add global
+ // permissions for the new user.
+ HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
+ try {
+ BlockingRpcChannel service = acl.coprocessorService(TEST_TABLE);
+ AccessControlService.BlockingInterface protocol =
+ AccessControlService.newBlockingStub(service);
+ String currentUser = User.getCurrent().getShortName();
+ // User name for the new RegionServer we plan to add.
+ String activeUserForNewRs = currentUser + ".hfs."
+ + hbaseCluster.getLiveRegionServerThreads().size();
+ ProtobufUtil.grant(protocol, activeUserForNewRs, null, null, null,
+ Permission.Action.ADMIN, Permission.Action.CREATE,
+ Permission.Action.READ, Permission.Action.WRITE);
+ } finally {
+ acl.close();
+ }
+ final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
+ HTableDescriptor htd = new HTableDescriptor(TEST_TABLE2);
+ htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
+ admin.createTable(htd);
+
+ // Starting a new RegionServer.
+ JVMClusterUtil.RegionServerThread newRsThread = hbaseCluster
+ .startRegionServer();
+ final HRegionServer newRs = newRsThread.getRegionServer();
+
+ // Move region to the new RegionServer.
+ final HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE2);
+ try {
+ NavigableMap<HRegionInfo, ServerName> regions = table
+ .getRegionLocations();
+ final Map.Entry<HRegionInfo, ServerName> firstRegion = regions.entrySet()
+ .iterator().next();
+
+ PrivilegedExceptionAction moveAction = new PrivilegedExceptionAction() {
+ public Object run() throws Exception {
+ admin.move(firstRegion.getKey().getEncodedNameAsBytes(),
+ Bytes.toBytes(newRs.getServerName().getServerName()));
+ return null;
+ }
+ };
+ SUPERUSER.runAs(moveAction);
+
+ final int RETRIES_LIMIT = 10;
+ int retries = 0;
+ while (newRs.getOnlineRegions(TEST_TABLE2).size() < 1 && retries < RETRIES_LIMIT)
{
+ LOG.debug("Waiting for region to be opened. Already retried " + retries
+ + " times.");
+ try {
+ Thread.sleep(200);
+ } catch (InterruptedException e) {
+ }
+ retries++;
+ if (retries == RETRIES_LIMIT - 1) {
+ fail("Retry exhaust for waiting region to be opened.");
+ }
+ }
+ // Verify write permission for user "admin2" who has the global
+ // permissions.
+ PrivilegedExceptionAction putAction = new PrivilegedExceptionAction() {
+ public Object run() throws Exception {
+ Put put = new Put(Bytes.toBytes("test"));
+ put.add(TEST_FAMILY, Bytes.toBytes("qual"), Bytes.toBytes("value"));
+ table.put(put);
+ return null;
+ }
+ };
+ USER_ADMIN.runAs(putAction);
+ } finally {
+ table.close();
+ }
+ }
}
|