HBASE-3025 Security: coprocessor based access control

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1203909 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Gary Helmling 2011-11-19 00:15:27 +00:00
parent a2de91f85b
commit cc7e469310
28 changed files with 4918 additions and 10 deletions

View File

@ -802,6 +802,7 @@ Release 0.92.0 - Unreleased
(Jonathan Hsieh)
HBASE-4298 Support to drain RS nodes through ZK (Aravind Gottipati)
HBASE-2742 Provide strong authentication with a secure RPC engine
HBASE-3025 Coprocessor based access control
Release 0.90.5 - Unreleased

View File

@ -0,0 +1,84 @@
/*
* 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 java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.filter.FilterBase;
import org.apache.hadoop.hbase.security.User;
/**
* <strong>NOTE: for internal use only by AccessController implementation</strong>
*
* <p>
* TODO: There is room for further performance optimization here.
* Calling TableAuthManager.authorize() per KeyValue imposes a fair amount of
* overhead. A more optimized solution might look at the qualifiers where
* permissions are actually granted and explicitly limit the scan to those.
* </p>
* <p>
* We should aim to use this _only_ when access to the requested column families
* is not granted at the column family levels. If table or column family
* access succeeds, then there is no need to impose the overhead of this filter.
* </p>
*/
class AccessControlFilter extends FilterBase {
private TableAuthManager authManager;
private byte[] table;
private User user;
/**
* For Writable
*/
AccessControlFilter() {
}
AccessControlFilter(TableAuthManager mgr, User ugi,
byte[] tableName) {
authManager = mgr;
table = tableName;
user = ugi;
}
@Override
public ReturnCode filterKeyValue(KeyValue kv) {
if (authManager.authorize(user, table, kv, TablePermission.Action.READ)) {
return ReturnCode.INCLUDE;
}
return ReturnCode.NEXT_COL;
}
@Override
public void write(DataOutput dataOutput) throws IOException {
// no implementation, server-side use only
throw new UnsupportedOperationException(
"Serialization not supported. Intended for server-side use only.");
}
@Override
public void readFields(DataInput dataInput) throws IOException {
// no implementation, server-side use only
throw new UnsupportedOperationException(
"Serialization not supported. Intended for server-side use only.");
}
}

View File

@ -0,0 +1,514 @@
/*
* 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 com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.HbaseObjectWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text;
import java.io.ByteArrayOutputStream;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.*;
/**
* Maintains lists of permission grants to users and groups to allow for
* authorization checks by {@link AccessController}.
*
* <p>
* Access control lists are stored in an "internal" metadata table named
* {@code _acl_}. Each table's permission grants are stored as a separate row,
* keyed by the table name. KeyValues for permissions assignments are stored
* in one of the formats:
* <pre>
* Key Desc
* -------- --------
* user table level permissions for a user [R=read, W=write]
* @group table level permissions for a group
* user,family column family level permissions for a user
* @group,family column family level permissions for a group
* user,family,qualifier column qualifier level permissions for a user
* @group,family,qualifier column qualifier level permissions for a group
* </pre>
* All values are encoded as byte arrays containing the codes from the
* {@link org.apache.hadoop.hbase.security.access.TablePermission.Action} enum.
* </p>
*/
public class AccessControlLists {
/** Internal storage table for access control lists */
public static final String ACL_TABLE_NAME_STR = "_acl_";
public static final byte[] ACL_TABLE_NAME = Bytes.toBytes(ACL_TABLE_NAME_STR);
/** Column family used to store ACL grants */
public static final String ACL_LIST_FAMILY_STR = "l";
public static final byte[] ACL_LIST_FAMILY = Bytes.toBytes(ACL_LIST_FAMILY_STR);
/** Table descriptor for ACL internal table */
public static final HTableDescriptor ACL_TABLEDESC = new HTableDescriptor(
ACL_TABLE_NAME);
static {
ACL_TABLEDESC.addFamily(
new HColumnDescriptor(ACL_LIST_FAMILY,
10, // Ten is arbitrary number. Keep versions to help debugging.
Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
HConstants.FOREVER, StoreFile.BloomType.NONE.toString(),
HConstants.REPLICATION_SCOPE_LOCAL));
}
/**
* Delimiter to separate user, column family, and qualifier in
* _acl_ table info: column keys */
public static final char ACL_KEY_DELIMITER = ',';
/** Prefix character to denote group names */
public static final String GROUP_PREFIX = "@";
/** Configuration key for superusers */
public static final String SUPERUSER_CONF_KEY = "hbase.superuser";
private static Log LOG = LogFactory.getLog(AccessControlLists.class);
/**
* Check for existence of {@code _acl_} table and create it if it does not exist
* @param master reference to HMaster
*/
static void init(MasterServices master) throws IOException {
if (!MetaReader.tableExists(master.getCatalogTracker(), ACL_TABLE_NAME_STR)) {
master.createTable(ACL_TABLEDESC, null);
}
}
/**
* Stores a new table permission grant in the access control lists table.
* @param conf the configuration
* @param tableName the table to which access is being granted
* @param username the user or group being granted the permission
* @param perm the details of the permission being granted
* @throws IOException in the case of an error accessing the metadata table
*/
static void addTablePermission(Configuration conf,
byte[] tableName, String username, TablePermission perm)
throws IOException {
Put p = new Put(tableName);
byte[] key = Bytes.toBytes(username);
if (perm.getFamily() != null && perm.getFamily().length > 0) {
key = Bytes.add(key,
Bytes.add(new byte[]{ACL_KEY_DELIMITER}, perm.getFamily()));
if (perm.getQualifier() != null && perm.getQualifier().length > 0) {
key = Bytes.add(key,
Bytes.add(new byte[]{ACL_KEY_DELIMITER}, perm.getQualifier()));
}
}
TablePermission.Action[] actions = perm.getActions();
if ((actions == null) || (actions.length == 0)) {
LOG.warn("No actions associated with user '"+username+"'");
return;
}
byte[] value = new byte[actions.length];
for (int i = 0; i < actions.length; i++) {
value[i] = actions[i].code();
}
p.add(ACL_LIST_FAMILY, key, value);
if (LOG.isDebugEnabled()) {
LOG.debug("Writing permission for table "+
Bytes.toString(tableName)+" "+
Bytes.toString(key)+": "+Bytes.toStringBinary(value)
);
}
HTable acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.put(p);
} finally {
if (acls != null) acls.close();
}
}
/**
* Removes a previously granted permission from the stored access control
* lists. The {@link TablePermission} being removed must exactly match what
* is stored -- no wildcard matching is attempted. Ie, if user "bob" has
* been granted "READ" access to the "data" table, but only to column family
* plus qualifier "info:colA", then trying to call this method with only
* user "bob" and the table name "data" (but without specifying the
* column qualifier "info:colA") will have no effect.
*
* @param conf the configuration
* @param tableName the table of the current permission grant
* @param userName the user or group currently granted the permission
* @param perm the details of the permission to be revoked
* @throws IOException if there is an error accessing the metadata table
*/
static void removeTablePermission(Configuration conf,
byte[] tableName, String userName, TablePermission perm)
throws IOException {
Delete d = new Delete(tableName);
byte[] key = null;
if (perm.getFamily() != null && perm.getFamily().length > 0) {
key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
Bytes.toString(perm.getFamily()));
if (perm.getQualifier() != null && perm.getQualifier().length > 0) {
key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
Bytes.toString(perm.getFamily()) + ACL_KEY_DELIMITER +
Bytes.toString(perm.getQualifier()));
} else {
key = Bytes.toBytes(userName + ACL_KEY_DELIMITER +
Bytes.toString(perm.getFamily()));
}
} else {
key = Bytes.toBytes(userName);
}
if (LOG.isDebugEnabled()) {
LOG.debug("Removing permission for user '" + userName+ "': "+
perm.toString());
}
d.deleteColumns(ACL_LIST_FAMILY, key);
HTable acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d);
} finally {
if (acls != null) acls.close();
}
}
/**
* Returns {@code true} if the given region is part of the {@code _acl_}
* metadata table.
*/
static boolean isAclRegion(HRegion region) {
return Bytes.equals(ACL_TABLE_NAME, region.getTableDesc().getName());
}
/**
* Loads all of the permission grants stored in a region of the {@code _acl_}
* table.
*
* @param aclRegion
* @return
* @throws IOException
*/
static Map<byte[],ListMultimap<String,TablePermission>> loadAll(
HRegion aclRegion)
throws IOException {
if (!isAclRegion(aclRegion)) {
throw new IOException("Can only load permissions from "+ACL_TABLE_NAME_STR);
}
Map<byte[],ListMultimap<String,TablePermission>> allPerms =
new TreeMap<byte[],ListMultimap<String,TablePermission>>(Bytes.BYTES_COMPARATOR);
// do a full scan of _acl_ table
Scan scan = new Scan();
scan.addFamily(ACL_LIST_FAMILY);
InternalScanner iScanner = null;
try {
iScanner = aclRegion.getScanner(scan);
while (true) {
List<KeyValue> row = new ArrayList<KeyValue>();
boolean hasNext = iScanner.next(row);
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
byte[] table = null;
for (KeyValue kv : row) {
if (table == null) {
table = kv.getRow();
}
Pair<String,TablePermission> permissionsOfUserOnTable =
parseTablePermissionRecord(table, kv);
if (permissionsOfUserOnTable != null) {
String username = permissionsOfUserOnTable.getFirst();
TablePermission permissions = permissionsOfUserOnTable.getSecond();
perms.put(username, permissions);
}
}
if (table != null) {
allPerms.put(table, perms);
}
if (!hasNext) {
break;
}
}
} finally {
if (iScanner != null) {
iScanner.close();
}
}
return allPerms;
}
/**
* Load all permissions from the region server holding {@code _acl_},
* primarily intended for testing purposes.
*/
static Map<byte[],ListMultimap<String,TablePermission>> loadAll(
Configuration conf) throws IOException {
Map<byte[],ListMultimap<String,TablePermission>> allPerms =
new TreeMap<byte[],ListMultimap<String,TablePermission>>(Bytes.BYTES_COMPARATOR);
// do a full scan of _acl_, filtering on only first table region rows
Scan scan = new Scan();
scan.addFamily(ACL_LIST_FAMILY);
HTable acls = null;
ResultScanner scanner = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
scanner = acls.getScanner(scan);
for (Result row : scanner) {
ListMultimap<String,TablePermission> resultPerms =
parseTablePermissions(row.getRow(), row);
allPerms.put(row.getRow(), resultPerms);
}
} finally {
if (scanner != null) scanner.close();
if (acls != null) acls.close();
}
return allPerms;
}
/**
* Reads user permission assignments stored in the <code>l:</code> column
* family of the first table row in <code>_acl_</code>.
*
* <p>
* See {@link AccessControlLists class documentation} for the key structure
* used for storage.
* </p>
*/
static ListMultimap<String,TablePermission> getTablePermissions(
Configuration conf, byte[] tableName)
throws IOException {
/* TODO: -ROOT- and .META. cannot easily be handled because they must be
* online before _acl_ table. Can anything be done here?
*/
if (Bytes.equals(tableName, HConstants.ROOT_TABLE_NAME) ||
Bytes.equals(tableName, HConstants.META_TABLE_NAME) ||
Bytes.equals(tableName, AccessControlLists.ACL_TABLE_NAME)) {
return ArrayListMultimap.create(0,0);
}
// for normal user tables, we just read the table row from _acl_
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
HTable acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
Get get = new Get(tableName);
get.addFamily(ACL_LIST_FAMILY);
Result row = acls.get(get);
if (!row.isEmpty()) {
perms = parseTablePermissions(tableName, row);
} else {
LOG.info("No permissions found in "+ACL_TABLE_NAME_STR+
" for table "+Bytes.toString(tableName));
}
} finally {
if (acls != null) acls.close();
}
return perms;
}
/**
* Returns the currently granted permissions for a given table as a list of
* user plus associated permissions.
*/
static List<UserPermission> getUserPermissions(
Configuration conf, byte[] tableName)
throws IOException {
ListMultimap<String,TablePermission> allPerms = getTablePermissions(
conf, tableName);
List<UserPermission> perms = new ArrayList<UserPermission>();
for (Map.Entry<String, TablePermission> entry : allPerms.entries()) {
UserPermission up = new UserPermission(Bytes.toBytes(entry.getKey()),
entry.getValue().getTable(), entry.getValue().getFamily(),
entry.getValue().getQualifier(), entry.getValue().getActions());
perms.add(up);
}
return perms;
}
private static ListMultimap<String,TablePermission> parseTablePermissions(
byte[] table, Result result) {
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
if (result != null && result.size() > 0) {
for (KeyValue kv : result.raw()) {
Pair<String,TablePermission> permissionsOfUserOnTable =
parseTablePermissionRecord(table, kv);
if (permissionsOfUserOnTable != null) {
String username = permissionsOfUserOnTable.getFirst();
TablePermission permissions = permissionsOfUserOnTable.getSecond();
perms.put(username, permissions);
}
}
}
return perms;
}
private static Pair<String,TablePermission> parseTablePermissionRecord(
byte[] table, KeyValue kv) {
// return X given a set of permissions encoded in the permissionRecord kv.
byte[] family = kv.getFamily();
if (!Bytes.equals(family, ACL_LIST_FAMILY)) {
return null;
}
byte[] key = kv.getQualifier();
byte[] value = kv.getValue();
if (LOG.isDebugEnabled()) {
LOG.debug("Read acl: kv ["+
Bytes.toStringBinary(key)+": "+
Bytes.toStringBinary(value)+"]");
}
// check for a column family appended to the key
// TODO: avoid the string conversion to make this more efficient
String username = Bytes.toString(key);
int idx = username.indexOf(ACL_KEY_DELIMITER);
byte[] permFamily = null;
byte[] permQualifier = null;
if (idx > 0 && idx < username.length()-1) {
String remainder = username.substring(idx+1);
username = username.substring(0, idx);
idx = remainder.indexOf(ACL_KEY_DELIMITER);
if (idx > 0 && idx < remainder.length()-1) {
permFamily = Bytes.toBytes(remainder.substring(0, idx));
permQualifier = Bytes.toBytes(remainder.substring(idx+1));
} else {
permFamily = Bytes.toBytes(remainder);
}
}
return new Pair<String,TablePermission>(
username, new TablePermission(table, permFamily, permQualifier, value));
}
/**
* Writes a set of permissions as {@link org.apache.hadoop.io.Writable} instances
* to the given output stream.
* @param out
* @param perms
* @param conf
* @throws IOException
*/
public static void writePermissions(DataOutput out,
ListMultimap<String,? extends Permission> perms, Configuration conf)
throws IOException {
Set<String> keys = perms.keySet();
out.writeInt(keys.size());
for (String key : keys) {
Text.writeString(out, key);
HbaseObjectWritable.writeObject(out, perms.get(key), List.class, conf);
}
}
/**
* Writes a set of permissions as {@link org.apache.hadoop.io.Writable} instances
* and returns the resulting byte array.
*/
public static byte[] writePermissionsAsBytes(
ListMultimap<String,? extends Permission> perms, Configuration conf) {
try {
ByteArrayOutputStream bos = new ByteArrayOutputStream();
writePermissions(new DataOutputStream(bos), perms, conf);
return bos.toByteArray();
} catch (IOException ioe) {
// shouldn't happen here
LOG.error("Error serializing permissions", ioe);
}
return null;
}
/**
* Reads a set of permissions as {@link org.apache.hadoop.io.Writable} instances
* from the input stream.
*/
public static <T extends Permission> ListMultimap<String,T> readPermissions(
DataInput in, Configuration conf) throws IOException {
ListMultimap<String,T> perms = ArrayListMultimap.create();
int length = in.readInt();
for (int i=0; i<length; i++) {
String user = Text.readString(in);
List<T> userPerms =
(List)HbaseObjectWritable.readObject(in, conf);
perms.putAll(user, userPerms);
}
return perms;
}
/**
* Returns whether or not the given name should be interpreted as a group
* principal. Currently this simply checks if the name starts with the
* special group prefix character ("@").
*/
public static boolean isGroupPrincipal(String name) {
return name != null && name.startsWith(GROUP_PREFIX);
}
/**
* Returns the actual name for a group principal (stripped of the
* group prefix).
*/
public static String getGroupName(String aclKey) {
if (!isGroupPrincipal(aclKey)) {
return aclKey;
}
return aclKey.substring(GROUP_PREFIX.length());
}
}

View File

@ -0,0 +1,987 @@
/*
* Licensed 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 com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import com.google.common.collect.MapMaker;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.MasterObserver;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.filter.CompareFilter;
import org.apache.hadoop.hbase.filter.FilterList;
import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
import org.apache.hadoop.hbase.ipc.HBaseRPC;
import org.apache.hadoop.hbase.ipc.ProtocolSignature;
import org.apache.hadoop.hbase.ipc.RequestContext;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.InternalScanner;
import org.apache.hadoop.hbase.regionserver.RegionScanner;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.IOException;
import java.util.*;
/**
* Provides basic authorization checks for data access and administrative
* operations.
*
* <p>
* {@code AccessController} performs authorization checks for HBase operations
* based on:
* <ul>
* <li>the identity of the user performing the operation</li>
* <li>the scope over which the operation is performed, in increasing
* specificity: global, table, column family, or qualifier</li>
* <li>the type of action being performed (as mapped to
* {@link Permission.Action} values)</li>
* </ul>
* If the authorization check fails, an {@link AccessDeniedException}
* will be thrown for the operation.
* </p>
*
* <p>
* To perform authorization checks, {@code AccessController} relies on the
* {@link org.apache.hadoop.hbase.ipc.SecureRpcEngine} being loaded to provide
* the user identities for remote requests.
* </p>
*
* <p>
* The access control lists used for authorization can be manipulated via the
* exposed {@link AccessControllerProtocol} implementation, and the associated
* {@code grant}, {@code revoke}, and {@code user_permission} HBase shell
* commands.
* </p>
*/
public class AccessController extends BaseRegionObserver
implements MasterObserver, AccessControllerProtocol {
/**
* Represents the result of an authorization check for logging and error
* reporting.
*/
private static class AuthResult {
private final boolean allowed;
private final byte[] table;
private final byte[] family;
private final byte[] qualifier;
private final Permission.Action action;
private final String reason;
private final User user;
public AuthResult(boolean allowed, String reason, User user,
Permission.Action action, byte[] table, byte[] family, byte[] qualifier) {
this.allowed = allowed;
this.reason = reason;
this.user = user;
this.table = table;
this.family = family;
this.qualifier = qualifier;
this.action = action;
}
public boolean isAllowed() { return allowed; }
public User getUser() { return user; }
public String getReason() { return reason; }
public String toContextString() {
return "(user=" + (user != null ? user.getName() : "UNKNOWN") + ", " +
"scope=" + (table == null ? "GLOBAL" : Bytes.toString(table)) + ", " +
"family=" + (family != null ? Bytes.toString(family) : "") + ", " +
"qualifer=" + (qualifier != null ? Bytes.toString(qualifier) : "") + ", " +
"action=" + (action != null ? action.toString() : "") + ")";
}
public String toString() {
return new StringBuilder("AuthResult")
.append(toContextString()).toString();
}
public static AuthResult allow(String reason, User user,
Permission.Action action, byte[] table) {
return new AuthResult(true, reason, user, action, table, null, null);
}
public static AuthResult deny(String reason, User user,
Permission.Action action, byte[] table) {
return new AuthResult(false, reason, user, action, table, null, null);
}
public static AuthResult deny(String reason, User user,
Permission.Action action, byte[] table, byte[] family, byte[] qualifier) {
return new AuthResult(false, reason, user, action, table, family, qualifier);
}
}
public static final Log LOG = LogFactory.getLog(AccessController.class);
private static final Log AUDITLOG =
LogFactory.getLog("SecurityLogger."+AccessController.class.getName());
/**
* Version number for AccessControllerProtocol
*/
private static final long PROTOCOL_VERSION = 1L;
TableAuthManager authManager = null;
// flags if we are running on a region of the _acl_ table
boolean aclRegion = false;
// defined only for Endpoint implementation, so it can have way to
// access region services.
private RegionCoprocessorEnvironment regionEnv;
/** Mapping of scanner instances to the user who created them */
private Map<InternalScanner,String> scannerOwners =
new MapMaker().weakKeys().makeMap();
void initialize(RegionCoprocessorEnvironment e) throws IOException {
final HRegion region = e.getRegion();
Map<byte[],ListMultimap<String,TablePermission>> tables =
AccessControlLists.loadAll(region);
// For each table, write out the table's permissions to the respective
// znode for that table.
for (Map.Entry<byte[],ListMultimap<String,TablePermission>> t:
tables.entrySet()) {
byte[] table = t.getKey();
String tableName = Bytes.toString(table);
ListMultimap<String,TablePermission> perms = t.getValue();
byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms,
e.getRegion().getConf());
this.authManager.getZKPermissionWatcher().writeToZookeeper(tableName,
serialized);
}
}
/**
* Writes all table ACLs for the tables in the given Map up into ZooKeeper
* znodes. This is called to synchronize ACL changes following {@code _acl_}
* table updates.
*/
void updateACL(RegionCoprocessorEnvironment e,
final Map<byte[], List<KeyValue>> familyMap) {
Set<String> tableSet = new HashSet<String>();
for (Map.Entry<byte[], List<KeyValue>> f : familyMap.entrySet()) {
List<KeyValue> kvs = f.getValue();
for (KeyValue kv: kvs) {
if (Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(),
kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
AccessControlLists.ACL_LIST_FAMILY.length) == 0) {
String tableName = Bytes.toString(kv.getRow());
tableSet.add(tableName);
}
}
}
for (String tableName: tableSet) {
try {
ListMultimap<String,TablePermission> perms =
AccessControlLists.getTablePermissions(regionEnv.getConfiguration(),
Bytes.toBytes(tableName));
byte[] serialized = AccessControlLists.writePermissionsAsBytes(
perms, e.getRegion().getConf());
this.authManager.getZKPermissionWatcher().writeToZookeeper(tableName,
serialized);
} catch (IOException ex) {
LOG.error("Failed updating permissions mirror for '" + tableName +
"'", ex);
}
}
}
/**
* Check the current user for authorization to perform a specific action
* against the given set of row data.
*
* <p>Note: Ordering of the authorization checks
* has been carefully optimized to short-circuit the most common requests
* and minimize the amount of processing required.</p>
*
* @param permRequest the action being requested
* @param e the coprocessor environment
* @param families the map of column families to qualifiers present in
* the request
* @return
*/
AuthResult permissionGranted(User user, TablePermission.Action permRequest,
RegionCoprocessorEnvironment e,
Map<byte [], ? extends Collection<?>> families) {
HRegionInfo hri = e.getRegion().getRegionInfo();
HTableDescriptor htd = e.getRegion().getTableDesc();
byte[] tableName = hri.getTableName();
// 1. All users need read access to .META. and -ROOT- tables.
// this is a very common operation, so deal with it quickly.
if ((hri.isRootRegion() || hri.isMetaRegion()) &&
(permRequest == TablePermission.Action.READ)) {
return AuthResult.allow("All users allowed", user, permRequest,
hri.getTableName());
}
if (user == null) {
return AuthResult.deny("No user associated with request!", null,
permRequest, hri.getTableName());
}
// 2. The table owner has full privileges
String owner = htd.getOwnerString();
if (user.getShortName().equals(owner)) {
// owner of the table has full access
return AuthResult.allow("User is table owner", user, permRequest,
hri.getTableName());
}
// 3. check for the table-level, if successful we can short-circuit
if (authManager.authorize(user, tableName, (byte[])null, permRequest)) {
return AuthResult.allow("Table permission granted", user,
permRequest, tableName);
}
// 4. check permissions against the requested families
if (families != null && families.size() > 0) {
// all families must pass
for (Map.Entry<byte [], ? extends Collection<?>> family : families.entrySet()) {
// a) check for family level access
if (authManager.authorize(user, tableName, family.getKey(),
permRequest)) {
continue; // family-level permission overrides per-qualifier
}
// b) qualifier level access can still succeed
if ((family.getValue() != null) && (family.getValue().size() > 0)) {
if (family.getValue() instanceof Set) {
// for each qualifier of the family
Set<byte[]> familySet = (Set<byte[]>)family.getValue();
for (byte[] qualifier : familySet) {
if (!authManager.authorize(user, tableName, family.getKey(),
qualifier, permRequest)) {
return AuthResult.deny("Failed qualifier check", user,
permRequest, tableName, family.getKey(), qualifier);
}
}
} else if (family.getValue() instanceof List) { // List<KeyValue>
List<KeyValue> kvList = (List<KeyValue>)family.getValue();
for (KeyValue kv : kvList) {
if (!authManager.authorize(user, tableName, family.getKey(),
kv.getQualifier(), permRequest)) {
return AuthResult.deny("Failed qualifier check", user,
permRequest, tableName, family.getKey(), kv.getQualifier());
}
}
}
} else {
// no qualifiers and family-level check already failed
return AuthResult.deny("Failed family check", user, permRequest,
tableName, family.getKey(), null);
}
}
// all family checks passed
return AuthResult.allow("All family checks passed", user, permRequest,
tableName);
}
// 5. no families to check and table level access failed
return AuthResult.deny("No families to check and table permission failed",
user, permRequest, tableName);
}
private void logResult(AuthResult result) {
if (AUDITLOG.isTraceEnabled()) {
AUDITLOG.trace("Access " + (result.isAllowed() ? "allowed" : "denied") +
" for user " + (result.getUser() != null ? result.getUser().getShortName() : "UNKNOWN") +
"; reason: " + result.getReason() +
"; context: " + result.toContextString());
}
}
/**
* Returns the active user to which authorization checks should be applied.
* If we are in the context of an RPC call, the remote user is used,
* otherwise the currently logged in user is used.
*/
private User getActiveUser() throws IOException {
User user = RequestContext.getRequestUser();
if (!RequestContext.isInRequestContext()) {
// for non-rpc handling, fallback to system user
user = User.getCurrent();
}
return user;
}
/**
* Authorizes that the current user has global privileges for the given action.
* @param perm The action being requested
* @throws IOException if obtaining the current user fails
* @throws AccessDeniedException if authorization is denied
*/
private void requirePermission(Permission.Action perm) throws IOException {
User user = getActiveUser();
if (authManager.authorize(user, perm)) {
logResult(AuthResult.allow("Global check allowed", user, perm, null));
} else {
logResult(AuthResult.deny("Global check failed", user, perm, null));
throw new AccessDeniedException("Insufficient permissions for user '" +
(user != null ? user.getShortName() : "null") +"' (global, action=" +
perm.toString() + ")");
}
}
/**
* Authorizes that the current user has permission to perform the given
* action on the set of table column families.
* @param perm Action that is required
* @param env The current coprocessor environment
* @param families The set of column families present/required in the request
* @throws AccessDeniedException if the authorization check failed
*/
private void requirePermission(Permission.Action perm,
RegionCoprocessorEnvironment env, Collection<byte[]> families)
throws IOException {
// create a map of family-qualifier
HashMap<byte[], Set<byte[]>> familyMap = new HashMap<byte[], Set<byte[]>>();
for (byte[] family : families) {
familyMap.put(family, null);
}
requirePermission(perm, env, familyMap);
}
/**
* Authorizes that the current user has permission to perform the given
* action on the set of table column families.
* @param perm Action that is required
* @param env The current coprocessor environment
* @param families The map of column families-qualifiers.
* @throws AccessDeniedException if the authorization check failed
*/
private void requirePermission(Permission.Action perm,
RegionCoprocessorEnvironment env,
Map<byte[], ? extends Collection<?>> families)
throws IOException {
User user = getActiveUser();
AuthResult result = permissionGranted(user, perm, env, families);
logResult(result);
if (!result.isAllowed()) {
StringBuffer sb = new StringBuffer("");
if ((families != null && families.size() > 0)) {
for (byte[] familyName : families.keySet()) {
if (sb.length() != 0) {
sb.append(", ");
}
sb.append(Bytes.toString(familyName));
}
}
throw new AccessDeniedException("Insufficient permissions (table=" +
env.getRegion().getTableDesc().getNameAsString()+
((families != null && families.size() > 0) ? ", family: " +
sb.toString() : "") + ", action=" +
perm.toString() + ")");
}
}
/**
* Returns <code>true</code> if the current user is allowed the given action
* over at least one of the column qualifiers in the given column families.
*/
private boolean hasFamilyQualifierPermission(User user,
TablePermission.Action perm,
RegionCoprocessorEnvironment env,
Map<byte[], ? extends Set<byte[]>> familyMap)
throws IOException {
HRegionInfo hri = env.getRegion().getRegionInfo();
byte[] tableName = hri.getTableName();
if (user == null) {
return false;
}
if (familyMap != null && familyMap.size() > 0) {
// at least one family must be allowed
for (Map.Entry<byte[], ? extends Set<byte[]>> family :
familyMap.entrySet()) {
if (family.getValue() != null && !family.getValue().isEmpty()) {
for (byte[] qualifier : family.getValue()) {
if (authManager.matchPermission(user, tableName,
family.getKey(), qualifier, perm)) {
return true;
}
}
} else {
if (authManager.matchPermission(user, tableName, family.getKey(),
perm)) {
return true;
}
}
}
} else if (LOG.isDebugEnabled()) {
LOG.debug("Empty family map passed for permission check");
}
return false;
}
/* ---- MasterObserver implementation ---- */
public void start(CoprocessorEnvironment env) throws IOException {
// if running on HMaster
if (env instanceof MasterCoprocessorEnvironment) {
MasterCoprocessorEnvironment e = (MasterCoprocessorEnvironment)env;
this.authManager = TableAuthManager.get(
e.getMasterServices().getZooKeeper(),
e.getConfiguration());
}
// if running at region
if (env instanceof RegionCoprocessorEnvironment) {
regionEnv = (RegionCoprocessorEnvironment)env;
}
}
public void stop(CoprocessorEnvironment env) {
}
@Override
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
requirePermission(Permission.Action.CREATE);
// default the table owner if not specified
User owner = getActiveUser();
if (desc.getOwnerString() == null ||
desc.getOwnerString().equals("")) {
desc.setOwner(owner);
}
}
@Override
public void postCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
HTableDescriptor desc, HRegionInfo[] regions) throws IOException {}
@Override
public void preDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {
requirePermission(Permission.Action.CREATE);
}
@Override
public void postDeleteTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {}
@Override
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HTableDescriptor htd) throws IOException {
requirePermission(Permission.Action.CREATE);
}
@Override
public void postModifyTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HTableDescriptor htd) throws IOException {}
@Override
public void preAddColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HColumnDescriptor column) throws IOException {
requirePermission(Permission.Action.CREATE);
}
@Override
public void postAddColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HColumnDescriptor column) throws IOException {}
@Override
public void preModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HColumnDescriptor descriptor) throws IOException {
requirePermission(Permission.Action.CREATE);
}
@Override
public void postModifyColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, HColumnDescriptor descriptor) throws IOException {}
@Override
public void preDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, byte[] col) throws IOException {
requirePermission(Permission.Action.CREATE);
}
@Override
public void postDeleteColumn(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName, byte[] col) throws IOException {}
@Override
public void preEnableTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {
/* TODO: Allow for users with global CREATE permission and the table owner */
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postEnableTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {}
@Override
public void preDisableTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {
/* TODO: Allow for users with global CREATE permission and the table owner */
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postDisableTable(ObserverContext<MasterCoprocessorEnvironment> c,
byte[] tableName) throws IOException {}
@Override
public void preMove(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo region, ServerName srcServer, ServerName destServer)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postMove(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo region, ServerName srcServer, ServerName destServer)
throws IOException {}
@Override
public void preAssign(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo) throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postAssign(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo) throws IOException {}
@Override
public void preUnassign(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo, boolean force) throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postUnassign(ObserverContext<MasterCoprocessorEnvironment> c,
HRegionInfo regionInfo, boolean force) throws IOException {}
@Override
public void preBalance(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postBalance(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {}
@Override
public boolean preBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
boolean newValue) throws IOException {
requirePermission(Permission.Action.ADMIN);
return newValue;
}
@Override
public void postBalanceSwitch(ObserverContext<MasterCoprocessorEnvironment> c,
boolean oldValue, boolean newValue) throws IOException {}
@Override
public void preShutdown(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void preStopMaster(ObserverContext<MasterCoprocessorEnvironment> c)
throws IOException {
requirePermission(Permission.Action.ADMIN);
}
@Override
public void postStartMaster(ObserverContext<MasterCoprocessorEnvironment> ctx)
throws IOException {
// initialize the ACL storage table
AccessControlLists.init(ctx.getEnvironment().getMasterServices());
}
/* ---- RegionObserver implementation ---- */
@Override
public void postOpen(ObserverContext<RegionCoprocessorEnvironment> c) {
RegionCoprocessorEnvironment e = c.getEnvironment();
final HRegion region = e.getRegion();
if (region == null) {
LOG.error("NULL region from RegionCoprocessorEnvironment in postOpen()");
return;
}
try {
this.authManager = TableAuthManager.get(
e.getRegionServerServices().getZooKeeper(),
e.getRegion().getConf());
} catch (IOException ioe) {
// pass along as a RuntimeException, so that the coprocessor is unloaded
throw new RuntimeException("Error obtaining TableAuthManager", ioe);
}
if (AccessControlLists.isAclRegion(region)) {
aclRegion = true;
try {
initialize(e);
} catch (IOException ex) {
// if we can't obtain permissions, it's better to fail
// than perform checks incorrectly
throw new RuntimeException("Failed to initialize permissions cache", ex);
}
}
}
@Override
public void preGetClosestRowBefore(final ObserverContext<RegionCoprocessorEnvironment> c,
final byte [] row, final byte [] family, final Result result)
throws IOException {
requirePermission(TablePermission.Action.READ, c.getEnvironment(),
(family != null ? Lists.newArrayList(family) : null));
}
@Override
public void preGet(final ObserverContext<RegionCoprocessorEnvironment> c,
final Get get, final List<KeyValue> result) throws IOException {
/*
if column family level checks fail, check for a qualifier level permission
in one of the families. If it is present, then continue with the AccessControlFilter.
*/
RegionCoprocessorEnvironment e = c.getEnvironment();
User requestUser = getActiveUser();
AuthResult authResult = permissionGranted(requestUser,
TablePermission.Action.READ, e, get.getFamilyMap());
if (!authResult.isAllowed()) {
if (hasFamilyQualifierPermission(requestUser,
TablePermission.Action.READ, e, get.getFamilyMap())) {
byte[] table = getTableName(e);
AccessControlFilter filter = new AccessControlFilter(authManager,
requestUser, table);
// wrap any existing filter
if (get.getFilter() != null) {
FilterList wrapper = new FilterList(FilterList.Operator.MUST_PASS_ALL,
Lists.newArrayList(filter, get.getFilter()));
get.setFilter(wrapper);
} else {
get.setFilter(filter);
}
logResult(AuthResult.allow("Access allowed with filter", requestUser,
TablePermission.Action.READ, authResult.table));
} else {
logResult(authResult);
throw new AccessDeniedException("Insufficient permissions (table=" +
e.getRegion().getTableDesc().getNameAsString() + ", action=READ)");
}
} else {
// log auth success
logResult(authResult);
}
}
@Override
public boolean preExists(final ObserverContext<RegionCoprocessorEnvironment> c,
final Get get, final boolean exists) throws IOException {
requirePermission(TablePermission.Action.READ, c.getEnvironment(),
get.familySet());
return exists;
}
@Override
public void prePut(final ObserverContext<RegionCoprocessorEnvironment> c,
final Put put, final WALEdit edit, final boolean writeToWAL)
throws IOException {
requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
put.getFamilyMap());
}
@Override
public void postPut(final ObserverContext<RegionCoprocessorEnvironment> c,
final Put put, final WALEdit edit, final boolean writeToWAL) {
if (aclRegion) {
updateACL(c.getEnvironment(), put.getFamilyMap());
}
}
@Override
public void preDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final Delete delete, final WALEdit edit, final boolean writeToWAL)
throws IOException {
requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
delete.getFamilyMap());
}
@Override
public void postDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final Delete delete, final WALEdit edit, final boolean writeToWAL)
throws IOException {
if (aclRegion) {
updateACL(c.getEnvironment(), delete.getFamilyMap());
}
}
@Override
public boolean preCheckAndPut(final ObserverContext<RegionCoprocessorEnvironment> c,
final byte [] row, final byte [] family, final byte [] qualifier,
final CompareFilter.CompareOp compareOp,
final WritableByteArrayComparable comparator, final Put put,
final boolean result) throws IOException {
requirePermission(TablePermission.Action.READ, c.getEnvironment(),
Arrays.asList(new byte[][]{family}));
return result;
}
@Override
public boolean preCheckAndDelete(final ObserverContext<RegionCoprocessorEnvironment> c,
final byte [] row, final byte [] family, final byte [] qualifier,
final CompareFilter.CompareOp compareOp,
final WritableByteArrayComparable comparator, final Delete delete,
final boolean result) throws IOException {
requirePermission(TablePermission.Action.READ, c.getEnvironment(),
Arrays.asList( new byte[][] {family}));
return result;
}
@Override
public long preIncrementColumnValue(final ObserverContext<RegionCoprocessorEnvironment> c,
final byte [] row, final byte [] family, final byte [] qualifier,
final long amount, final boolean writeToWAL)
throws IOException {
requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
Arrays.asList(new byte[][]{family}));
return -1;
}
@Override
public void preIncrement(final ObserverContext<RegionCoprocessorEnvironment> c,
final Increment increment, final Result result)
throws IOException {
requirePermission(TablePermission.Action.WRITE, c.getEnvironment(),
increment.getFamilyMap().keySet());
}
@Override
public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
final Scan scan, final RegionScanner s) throws IOException {
/*
if column family level checks fail, check for a qualifier level permission
in one of the families. If it is present, then continue with the AccessControlFilter.
*/
RegionCoprocessorEnvironment e = c.getEnvironment();
User user = getActiveUser();
AuthResult authResult = permissionGranted(user, TablePermission.Action.READ, e,
scan.getFamilyMap());
if (!authResult.isAllowed()) {
if (hasFamilyQualifierPermission(user, TablePermission.Action.READ, e,
scan.getFamilyMap())) {
byte[] table = getTableName(e);
AccessControlFilter filter = new AccessControlFilter(authManager,
user, table);
// wrap any existing filter
if (scan.hasFilter()) {
FilterList wrapper = new FilterList(FilterList.Operator.MUST_PASS_ALL,
Lists.newArrayList(filter, scan.getFilter()));
scan.setFilter(wrapper);
} else {
scan.setFilter(filter);
}
logResult(AuthResult.allow("Access allowed with filter", user,
TablePermission.Action.READ, authResult.table));
} else {
// no table/family level perms and no qualifier level perms, reject
logResult(authResult);
throw new AccessDeniedException("Insufficient permissions for user '"+
(user != null ? user.getShortName() : "null")+"' "+
"for scanner open on table " + Bytes.toString(getTableName(e)));
}
} else {
// log success
logResult(authResult);
}
return s;
}
@Override
public RegionScanner postScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> c,
final Scan scan, final RegionScanner s) throws IOException {
User user = getActiveUser();
if (user != null && user.getShortName() != null) { // store reference to scanner owner for later checks
scannerOwners.put(s, user.getShortName());
}
return s;
}
@Override
public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> c,
final InternalScanner s, final List<Result> result,
final int limit, final boolean hasNext) throws IOException {
requireScannerOwner(s);
return hasNext;
}
@Override
public void preScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
final InternalScanner s) throws IOException {
requireScannerOwner(s);
}
@Override
public void postScannerClose(final ObserverContext<RegionCoprocessorEnvironment> c,
final InternalScanner s) throws IOException {
// clean up any associated owner mapping
scannerOwners.remove(s);
}
/**
* Verify, when servicing an RPC, that the caller is the scanner owner.
* If so, we assume that access control is correctly enforced based on
* the checks performed in preScannerOpen()
*/
private void requireScannerOwner(InternalScanner s)
throws AccessDeniedException {
if (RequestContext.isInRequestContext()) {
String owner = scannerOwners.get(s);
if (owner != null && !owner.equals(RequestContext.getRequestUserName())) {
throw new AccessDeniedException("User '"+
RequestContext.getRequestUserName()+"' is not the scanner owner!");
}
}
}
/* ---- AccessControllerProtocol implementation ---- */
/*
* These methods are only allowed to be called against the _acl_ region(s).
* This will be restricted by both client side and endpoint implementations.
*/
@Override
public void grant(byte[] user, TablePermission permission)
throws IOException {
// verify it's only running at .acl.
if (aclRegion) {
if (LOG.isDebugEnabled()) {
LOG.debug("Received request to grant access permission to '"
+ Bytes.toString(user) + "'. "
+ permission.toString());
}
requirePermission(Permission.Action.ADMIN);
AccessControlLists.addTablePermission(regionEnv.getConfiguration(),
permission.getTable(), Bytes.toString(user), permission);
if (AUDITLOG.isTraceEnabled()) {
// audit log should store permission changes in addition to auth results
AUDITLOG.trace("Granted user '" + Bytes.toString(user) + "' permission "
+ permission.toString());
}
} else {
throw new CoprocessorException(AccessController.class, "This method " +
"can only execute at " +
Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
}
}
@Override
public void revoke(byte[] user, TablePermission permission)
throws IOException{
// only allowed to be called on _acl_ region
if (aclRegion) {
if (LOG.isDebugEnabled()) {
LOG.debug("Received request to revoke access permission for '"
+ Bytes.toString(user) + "'. "
+ permission.toString());
}
requirePermission(Permission.Action.ADMIN);
AccessControlLists.removeTablePermission(regionEnv.getConfiguration(),
permission.getTable(), Bytes.toString(user), permission);
if (AUDITLOG.isTraceEnabled()) {
// audit log should record all permission changes
AUDITLOG.trace("Revoked user '" + Bytes.toString(user) + "' permission "
+ permission.toString());
}
} else {
throw new CoprocessorException(AccessController.class, "This method " +
"can only execute at " +
Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
}
}
@Override
public List<UserPermission> getUserPermissions(final byte[] tableName)
throws IOException {
// only allowed to be called on _acl_ region
if (aclRegion) {
requirePermission(Permission.Action.ADMIN);
List<UserPermission> perms = AccessControlLists.getUserPermissions
(regionEnv.getConfiguration(), tableName);
return perms;
} else {
throw new CoprocessorException(AccessController.class, "This method " +
"can only execute at " +
Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
}
}
@Override
public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
return PROTOCOL_VERSION;
}
@Override
public ProtocolSignature getProtocolSignature(String protocol,
long clientVersion, int clientMethodsHash) throws IOException {
if (AccessControllerProtocol.class.getName().equals(protocol)) {
return new ProtocolSignature(PROTOCOL_VERSION, null);
}
throw new HBaseRPC.UnknownProtocolException(
"Unexpected protocol requested: "+protocol);
}
private byte[] getTableName(RegionCoprocessorEnvironment e) {
HRegion region = e.getRegion();
byte[] tableName = null;
if (region != null) {
HRegionInfo regionInfo = region.getRegionInfo();
if (regionInfo != null) {
tableName = regionInfo.getTableName();
}
}
return tableName;
}
}

View File

@ -0,0 +1,68 @@
/*
* 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 org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
import java.io.IOException;
import java.util.List;
/**
* A custom protocol defined for maintaining and querying access control lists.
*/
public interface AccessControllerProtocol extends CoprocessorProtocol {
/**
* Grants the given user or group the privilege to perform the given actions
* over the specified scope contained in {@link TablePermission}
* @param user the user name, or, if prefixed with "@", group name receiving
* the grant
* @param permission the details of the provided permissions
* @throws IOException if the grant could not be applied
*/
public void grant(byte[] user, TablePermission permission)
throws IOException;
/**
* Revokes a previously granted privilege from a user or group.
* Note that the provided {@link TablePermission} details must exactly match
* a stored grant. For example, if user "bob" has been granted "READ" access
* to table "data", over column family and qualifer "info:colA", then the
* table, column family and column qualifier must all be specified.
* Attempting to revoke permissions over just the "data" table will have
* no effect.
* @param user the user name, or, if prefixed with "@", group name whose
* privileges are being revoked
* @param permission the details of the previously granted permission to revoke
* @throws IOException if the revocation could not be performed
*/
public void revoke(byte[] user, TablePermission permission)
throws IOException;
/**
* Queries the permissions currently stored for the given table, returning
* a list of currently granted permissions, along with the user or group
* each is associated with.
* @param tableName the table of the permission grants to return
* @return a list of the currently granted permissions, with associated user
* or group names
* @throws IOException if there is an error querying the permissions
*/
public List<UserPermission> getUserPermissions(byte[] tableName)
throws IOException;
}

View File

@ -0,0 +1,199 @@
/*
* 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 com.google.common.collect.Maps;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.VersionedWritable;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
import java.util.Map;
/**
* Base permissions instance representing the ability to perform a given set
* of actions.
*
* @see TablePermission
*/
public class Permission extends VersionedWritable {
protected static final byte VERSION = 0;
public enum Action {
READ('R'), WRITE('W'), EXEC('X'), CREATE('C'), ADMIN('A');
private byte code;
Action(char code) {
this.code = (byte)code;
}
public byte code() { return code; }
}
private static Log LOG = LogFactory.getLog(Permission.class);
protected static Map<Byte,Action> ACTION_BY_CODE = Maps.newHashMap();
protected Action[] actions;
static {
for (Action a : Action.values()) {
ACTION_BY_CODE.put(a.code(), a);
}
}
/** Empty constructor for Writable implementation. <b>Do not use.</b> */
public Permission() {
super();
}
public Permission(Action... assigned) {
if (assigned != null && assigned.length > 0) {
actions = Arrays.copyOf(assigned, assigned.length);
}
}
public Permission(byte[] actionCodes) {
if (actionCodes != null) {
Action acts[] = new Action[actionCodes.length];
int j = 0;
for (int i=0; i<actionCodes.length; i++) {
byte b = actionCodes[i];
Action a = ACTION_BY_CODE.get(b);
if (a == null) {
LOG.error("Ignoring unknown action code '"+
Bytes.toStringBinary(new byte[]{b})+"'");
continue;
}
acts[j++] = a;
}
this.actions = Arrays.copyOf(acts, j);
}
}
public Action[] getActions() {
return actions;
}
public boolean implies(Action action) {
if (this.actions != null) {
for (Action a : this.actions) {
if (a == action) {
return true;
}
}
}
return false;
}
@Override
public boolean equals(Object obj) {
if (!(obj instanceof Permission)) {
return false;
}
Permission other = (Permission)obj;
// check actions
if (actions == null && other.getActions() == null) {
return true;
} else if (actions != null && other.getActions() != null) {
Action[] otherActions = other.getActions();
if (actions.length != otherActions.length) {
return false;
}
outer:
for (Action a : actions) {
for (Action oa : otherActions) {
if (a == oa) continue outer;
}
return false;
}
return true;
}
return false;
}
@Override
public int hashCode() {
final int prime = 37;
int result = 23;
for (Action a : actions) {
result = prime * result + a.code();
}
return result;
}
public String toString() {
StringBuilder str = new StringBuilder("[Permission: ")
.append("actions=");
if (actions != null) {
for (int i=0; i<actions.length; i++) {
if (i > 0)
str.append(",");
if (actions[i] != null)
str.append(actions[i].toString());
else
str.append("NULL");
}
}
str.append("]");
return str.toString();
}
/** @return the object version number */
public byte getVersion() {
return VERSION;
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
int length = (int)in.readByte();
if (length > 0) {
actions = new Action[length];
for (int i = 0; i < length; i++) {
byte b = in.readByte();
Action a = ACTION_BY_CODE.get(b);
if (a == null) {
throw new IOException("Unknown action code '"+
Bytes.toStringBinary(new byte[]{b})+"' in input");
}
this.actions[i] = a;
}
} else {
actions = new Action[0];
}
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
out.writeByte(actions != null ? actions.length : 0);
if (actions != null) {
for (Action a: actions) {
out.writeByte(a.code());
}
}
}
}

View File

@ -0,0 +1,482 @@
/*
* 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 com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Lists;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.*;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentSkipListMap;
/**
* Performs authorization checks for a given user's assigned permissions
*/
public class TableAuthManager {
/** Key for the user and group cache maps for globally assigned permissions */
private static final String GLOBAL_CACHE_KEY = ".access.";
private static Log LOG = LogFactory.getLog(TableAuthManager.class);
private static TableAuthManager instance;
/** Cache of global user permissions */
private ListMultimap<String,Permission> USER_CACHE = ArrayListMultimap.create();
/** Cache of global group permissions */
private ListMultimap<String,Permission> GROUP_CACHE = ArrayListMultimap.create();
private ConcurrentSkipListMap<byte[], ListMultimap<String,TablePermission>> TABLE_USER_CACHE =
new ConcurrentSkipListMap<byte[], ListMultimap<String,TablePermission>>(Bytes.BYTES_COMPARATOR);
private ConcurrentSkipListMap<byte[], ListMultimap<String,TablePermission>> TABLE_GROUP_CACHE =
new ConcurrentSkipListMap<byte[], ListMultimap<String,TablePermission>>(Bytes.BYTES_COMPARATOR);
private Configuration conf;
private ZKPermissionWatcher zkperms;
private TableAuthManager(ZooKeeperWatcher watcher, Configuration conf)
throws IOException {
this.conf = 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
initGlobal(conf);
}
private void initGlobal(Configuration conf) throws IOException {
User user = User.getCurrent();
if (user == null) {
throw new IOException("Unable to obtain the current user, " +
"authorization checks for internal operations will not work correctly!");
}
String currentUser = user.getShortName();
// the system user is always included
List<String> superusers = Lists.asList(currentUser, conf.getStrings(
AccessControlLists.SUPERUSER_CONF_KEY, new String[0]));
if (superusers != null) {
for (String name : superusers) {
if (AccessControlLists.isGroupPrincipal(name)) {
GROUP_CACHE.put(AccessControlLists.getGroupName(name),
new Permission(Permission.Action.values()));
} else {
USER_CACHE.put(name, new Permission(Permission.Action.values()));
}
}
}
}
public ZKPermissionWatcher getZKPermissionWatcher() {
return this.zkperms;
}
public void refreshCacheFromWritable(byte[] table, byte[] data) throws IOException {
if (data != null && data.length > 0) {
DataInput in = new DataInputStream( new ByteArrayInputStream(data) );
ListMultimap<String,TablePermission> perms = AccessControlLists.readPermissions(in, conf);
cache(table, perms);
} else {
LOG.debug("Skipping permission cache refresh because writable data is empty");
}
}
/**
* Updates the internal permissions cache for a single table, splitting
* the permissions listed into separate caches for users and groups to optimize
* group lookups.
*
* @param table
* @param tablePerms
*/
private void cache(byte[] table,
ListMultimap<String,TablePermission> tablePerms) {
// split user from group assignments so we don't have to prepend the group
// prefix every time we query for groups
ListMultimap<String,TablePermission> userPerms = ArrayListMultimap.create();
ListMultimap<String,TablePermission> groupPerms = ArrayListMultimap.create();
if (tablePerms != null) {
for (Map.Entry<String,TablePermission> entry : tablePerms.entries()) {
if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
groupPerms.put(
entry.getKey().substring(AccessControlLists.GROUP_PREFIX.length()),
entry.getValue());
} else {
userPerms.put(entry.getKey(), entry.getValue());
}
}
TABLE_GROUP_CACHE.put(table, groupPerms);
TABLE_USER_CACHE.put(table, userPerms);
}
}
private List<TablePermission> getUserPermissions(String username, byte[] table) {
ListMultimap<String, TablePermission> tablePerms = TABLE_USER_CACHE.get(table);
if (tablePerms != null) {
return tablePerms.get(username);
}
return null;
}
private List<TablePermission> getGroupPermissions(String groupName, byte[] table) {
ListMultimap<String, TablePermission> tablePerms = TABLE_GROUP_CACHE.get(table);
if (tablePerms != null) {
return tablePerms.get(groupName);
}
return null;
}
/**
* Authorizes a global permission
* @param perms
* @param action
* @return
*/
private boolean authorize(List<Permission> perms, Permission.Action action) {
if (perms != null) {
for (Permission p : perms) {
if (p.implies(action)) {
return true;
}
}
} else if (LOG.isDebugEnabled()) {
LOG.debug("No permissions found");
}
return false;
}
/**
* Authorize a global permission based on ACLs for the given user and the
* user's groups.
* @param user
* @param action
* @return
*/
public boolean authorize(User user, Permission.Action action) {
if (user == null) {
return false;
}
if (authorize(USER_CACHE.get(user.getShortName()), action)) {
return true;
}
String[] groups = user.getGroupNames();
if (groups != null) {
for (String group : groups) {
if (authorize(GROUP_CACHE.get(group), action)) {
return true;
}
}
}
return false;
}
private boolean authorize(List<TablePermission> perms, byte[] table, byte[] family,
Permission.Action action) {
return authorize(perms, table, family, null, action);
}
private boolean authorize(List<TablePermission> perms, byte[] table, byte[] family,
byte[] qualifier, Permission.Action action) {
if (perms != null) {
for (TablePermission p : perms) {
if (p.implies(table, family, qualifier, action)) {
return true;
}
}
} else if (LOG.isDebugEnabled()) {
LOG.debug("No permissions found for table="+Bytes.toStringBinary(table));
}
return false;
}
public boolean authorize(User user, byte[] table, KeyValue kv,
TablePermission.Action action) {
List<TablePermission> userPerms = getUserPermissions(
user.getShortName(), table);
if (authorize(userPerms, table, kv, action)) {
return true;
}
String[] groupNames = user.getGroupNames();
if (groupNames != null) {
for (String group : groupNames) {
List<TablePermission> groupPerms = getGroupPermissions(group, table);
if (authorize(groupPerms, table, kv, action)) {
return true;
}
}
}
return false;
}
private boolean authorize(List<TablePermission> perms, byte[] table, KeyValue kv,
TablePermission.Action action) {
if (perms != null) {
for (TablePermission p : perms) {
if (p.implies(table, kv, action)) {
return true;
}
}
} else if (LOG.isDebugEnabled()) {
LOG.debug("No permissions for authorize() check, table=" +
Bytes.toStringBinary(table));
}
return false;
}
/**
* Checks global authorization for a specific action for a user, based on the
* stored user permissions.
*/
public boolean authorizeUser(String username, Permission.Action action) {
return authorize(USER_CACHE.get(username), action);
}
/**
* Checks authorization to a given table and column family for a user, based on the
* stored user permissions.
*
* @param username
* @param table
* @param family
* @param action
* @return
*/
public boolean authorizeUser(String username, byte[] table, byte[] family,
Permission.Action action) {
return authorizeUser(username, table, family, null, action);
}
public boolean authorizeUser(String username, byte[] table, byte[] family,
byte[] qualifier, Permission.Action action) {
// global authorization supercedes table level
if (authorizeUser(username, action)) {
return true;
}
return authorize(getUserPermissions(username, table), table, family,
qualifier, action);
}
/**
* Checks authorization for a given action for a group, based on the stored
* permissions.
*/
public boolean authorizeGroup(String groupName, Permission.Action action) {
return authorize(GROUP_CACHE.get(groupName), action);
}
/**
* Checks authorization to a given table and column family for a group, based
* on the stored permissions.
* @param groupName
* @param table
* @param family
* @param action
* @return
*/
public boolean authorizeGroup(String groupName, byte[] table, byte[] family,
Permission.Action action) {
// global authorization supercedes table level
if (authorizeGroup(groupName, action)) {
return true;
}
return authorize(getGroupPermissions(groupName, table), table, family, action);
}
public boolean authorize(User user, byte[] table, byte[] family,
byte[] qualifier, Permission.Action action) {
if (authorizeUser(user.getShortName(), table, family, qualifier, action)) {
return true;
}
String[] groups = user.getGroupNames();
if (groups != null) {
for (String group : groups) {
if (authorizeGroup(group, table, family, action)) {
return true;
}
}
}
return false;
}
public boolean authorize(User user, byte[] table, byte[] family,
Permission.Action action) {
return authorize(user, table, family, null, action);
}
/**
* Returns true if the given user has a {@link TablePermission} matching up
* to the column family portion of a permission. Note that this permission
* may be scoped to a given column qualifier and does not guarantee that
* authorize() on the same column family would return true.
*/
public boolean matchPermission(User user,
byte[] table, byte[] family, TablePermission.Action action) {
List<TablePermission> userPerms = getUserPermissions(
user.getShortName(), table);
if (userPerms != null) {
for (TablePermission p : userPerms) {
if (p.matchesFamily(table, family, action)) {
return true;
}
}
}
String[] groups = user.getGroupNames();
if (groups != null) {
for (String group : groups) {
List<TablePermission> groupPerms = getGroupPermissions(group, table);
if (groupPerms != null) {
for (TablePermission p : groupPerms) {
if (p.matchesFamily(table, family, action)) {
return true;
}
}
}
}
}
return false;
}
public boolean matchPermission(User user,
byte[] table, byte[] family, byte[] qualifier,
TablePermission.Action action) {
List<TablePermission> userPerms = getUserPermissions(
user.getShortName(), table);
if (userPerms != null) {
for (TablePermission p : userPerms) {
if (p.matchesFamilyQualifier(table, family, qualifier, action)) {
return true;
}
}
}
String[] groups = user.getGroupNames();
if (groups != null) {
for (String group : groups) {
List<TablePermission> groupPerms = getGroupPermissions(group, table);
if (groupPerms != null) {
for (TablePermission p : groupPerms) {
if (p.matchesFamilyQualifier(table, family, qualifier, action)) {
return true;
}
}
}
}
}
return false;
}
public void remove(byte[] table) {
TABLE_USER_CACHE.remove(table);
TABLE_GROUP_CACHE.remove(table);
}
/**
* Overwrites the existing permission set for a given user for a table, and
* triggers an update for zookeeper synchronization.
* @param username
* @param table
* @param perms
*/
public void setUserPermissions(String username, byte[] table,
List<TablePermission> perms) {
ListMultimap<String,TablePermission> tablePerms = TABLE_USER_CACHE.get(table);
if (tablePerms == null) {
tablePerms = ArrayListMultimap.create();
TABLE_USER_CACHE.put(table, tablePerms);
}
tablePerms.replaceValues(username, perms);
writeToZooKeeper(table, tablePerms, TABLE_GROUP_CACHE.get(table));
}
/**
* Overwrites the existing permission set for a group and triggers an update
* for zookeeper synchronization.
* @param group
* @param table
* @param perms
*/
public void setGroupPermissions(String group, byte[] table,
List<TablePermission> perms) {
ListMultimap<String,TablePermission> tablePerms = TABLE_GROUP_CACHE.get(table);
if (tablePerms == null) {
tablePerms = ArrayListMultimap.create();
TABLE_GROUP_CACHE.put(table, tablePerms);
}
tablePerms.replaceValues(group, perms);
writeToZooKeeper(table, TABLE_USER_CACHE.get(table), tablePerms);
}
public void writeToZooKeeper(byte[] table,
ListMultimap<String,TablePermission> userPerms,
ListMultimap<String,TablePermission> groupPerms) {
ListMultimap<String,TablePermission> tmp = ArrayListMultimap.create();
if (userPerms != null) {
tmp.putAll(userPerms);
}
if (groupPerms != null) {
for (String group : groupPerms.keySet()) {
tmp.putAll(AccessControlLists.GROUP_PREFIX + group,
groupPerms.get(group));
}
}
byte[] serialized = AccessControlLists.writePermissionsAsBytes(tmp, conf);
zkperms.writeToZookeeper(Bytes.toString(table), serialized);
}
static Map<ZooKeeperWatcher,TableAuthManager> managerMap =
new HashMap<ZooKeeperWatcher,TableAuthManager>();
public synchronized static TableAuthManager get(
ZooKeeperWatcher watcher, Configuration conf) throws IOException {
instance = managerMap.get(watcher);
if (instance == null) {
instance = new TableAuthManager(watcher, conf);
managerMap.put(watcher, instance);
}
return instance;
}
}

View File

@ -0,0 +1,295 @@
/*
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
/**
* Represents an authorization for access for the given actions, optionally
* restricted to the given column family or column qualifier, over the
* given table. If the family property is <code>null</code>, it implies
* full table access.
*/
public class TablePermission extends Permission {
private static Log LOG = LogFactory.getLog(TablePermission.class);
private byte[] table;
private byte[] family;
private byte[] qualifier;
/** Nullary constructor for Writable, do not use */
public TablePermission() {
super();
}
/**
* Create a new permission for the given table and (optionally) column family,
* allowing the given actions.
* @param table the table
* @param family the family, can be null if a global permission on the table
* @param assigned the list of allowed actions
*/
public TablePermission(byte[] table, byte[] family, Action... assigned) {
this(table, family, null, assigned);
}
/**
* Creates a new permission for the given table, restricted to the given
* column family and qualifer, allowing the assigned actions to be performed.
* @param table the table
* @param family the family, can be null if a global permission on the table
* @param assigned the list of allowed actions
*/
public TablePermission(byte[] table, byte[] family, byte[] qualifier,
Action... assigned) {
super(assigned);
this.table = table;
this.family = family;
this.qualifier = qualifier;
}
/**
* Creates a new permission for the given table, family and column qualifier,
* allowing the actions matching the provided byte codes to be performed.
* @param table the table
* @param family the family, can be null if a global permission on the table
* @param actionCodes the list of allowed action codes
*/
public TablePermission(byte[] table, byte[] family, byte[] qualifier,
byte[] actionCodes) {
super(actionCodes);
this.table = table;
this.family = family;
this.qualifier = qualifier;
}
public byte[] getTable() {
return table;
}
public byte[] getFamily() {
return family;
}
public byte[] getQualifier() {
return qualifier;
}
/**
* Checks that a given table operation is authorized by this permission
* instance.
*
* @param table the table where the operation is being performed
* @param family the column family to which the operation is restricted,
* if <code>null</code> implies "all"
* @param qualifier the column qualifier to which the action is restricted,
* if <code>null</code> implies "all"
* @param action the action being requested
* @return <code>true</code> if the action within the given scope is allowed
* by this permission, <code>false</code>
*/
public boolean implies(byte[] table, byte[] family, byte[] qualifier,
Action action) {
if (!Bytes.equals(this.table, table)) {
return false;
}
if (this.family != null &&
(family == null ||
!Bytes.equals(this.family, family))) {
return false;
}
if (this.qualifier != null &&
(qualifier == null ||
!Bytes.equals(this.qualifier, qualifier))) {
return false;
}
// check actions
return super.implies(action);
}
/**
* Checks if this permission grants access to perform the given action on
* the given table and key value.
* @param table the table on which the operation is being performed
* @param kv the KeyValue on which the operation is being requested
* @param action the action requested
* @return <code>true</code> if the action is allowed over the given scope
* by this permission, otherwise <code>false</code>
*/
public boolean implies(byte[] table, KeyValue kv, Action action) {
if (!Bytes.equals(this.table, table)) {
return false;
}
if (family != null &&
(Bytes.compareTo(family, 0, family.length,
kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength()) != 0)) {
return false;
}
if (qualifier != null &&
(Bytes.compareTo(qualifier, 0, qualifier.length,
kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) != 0)) {
return false;
}
// check actions
return super.implies(action);
}
/**
* Returns <code>true</code> if this permission matches the given column
* family at least. This only indicates a partial match against the table
* and column family, however, and does not guarantee that implies() for the
* column same family would return <code>true</code>. In the case of a
* column-qualifier specific permission, for example, implies() would still
* return false.
*/
public boolean matchesFamily(byte[] table, byte[] family, Action action) {
if (!Bytes.equals(this.table, table)) {
return false;
}
if (this.family != null &&
(family == null ||
!Bytes.equals(this.family, family))) {
return false;
}
// ignore qualifier
// check actions
return super.implies(action);
}
/**
* Returns if the given permission matches the given qualifier.
* @param table the table name to match
* @param family the column family to match
* @param qualifier the qualifier name to match
* @param action the action requested
* @return <code>true</code> if the table, family and qualifier match,
* otherwise <code>false</code>
*/
public boolean matchesFamilyQualifier(byte[] table, byte[] family, byte[] qualifier,
Action action) {
if (!matchesFamily(table, family, action)) {
return false;
} else {
if (this.qualifier != null &&
(qualifier == null ||
!Bytes.equals(this.qualifier, qualifier))) {
return false;
}
}
return super.implies(action);
}
@Override
public boolean equals(Object obj) {
if (!(obj instanceof TablePermission)) {
return false;
}
TablePermission other = (TablePermission)obj;
if (!(Bytes.equals(table, other.getTable()) &&
((family == null && other.getFamily() == null) ||
Bytes.equals(family, other.getFamily())) &&
((qualifier == null && other.getQualifier() == null) ||
Bytes.equals(qualifier, other.getQualifier()))
)) {
return false;
}
// check actions
return super.equals(other);
}
@Override
public int hashCode() {
final int prime = 37;
int result = super.hashCode();
if (table != null) {
result = prime * result + Bytes.hashCode(table);
}
if (family != null) {
result = prime * result + Bytes.hashCode(family);
}
if (qualifier != null) {
result = prime * result + Bytes.hashCode(qualifier);
}
return result;
}
public String toString() {
StringBuilder str = new StringBuilder("[TablePermission: ")
.append("table=").append(Bytes.toString(table))
.append(", family=").append(Bytes.toString(family))
.append(", qualifier=").append(Bytes.toString(qualifier))
.append(", actions=");
if (actions != null) {
for (int i=0; i<actions.length; i++) {
if (i > 0)
str.append(",");
if (actions[i] != null)
str.append(actions[i].toString());
else
str.append("NULL");
}
}
str.append("]");
return str.toString();
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
table = Bytes.readByteArray(in);
if (in.readBoolean()) {
family = Bytes.readByteArray(in);
}
if (in.readBoolean()) {
qualifier = Bytes.readByteArray(in);
}
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Bytes.writeByteArray(out, table);
out.writeBoolean(family != null);
if (family != null) {
Bytes.writeByteArray(out, family);
}
out.writeBoolean(qualifier != null);
if (qualifier != null) {
Bytes.writeByteArray(out, qualifier);
}
}
}

View File

@ -0,0 +1,138 @@
/*
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.util.Bytes;
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
/**
* Represents an authorization for access over the given table, column family
* plus qualifier, for the given user.
*/
public class UserPermission extends TablePermission {
private static Log LOG = LogFactory.getLog(UserPermission.class);
private byte[] user;
/** Nullary constructor for Writable, do not use */
public UserPermission() {
super();
}
/**
* Creates a new instance for the given user, table and column family.
* @param user the user
* @param table the table
* @param family the family, can be null if action is allowed over the entire
* table
* @param assigned the list of allowed actions
*/
public UserPermission(byte[] user, byte[] table, byte[] family,
Action... assigned) {
super(table, family, assigned);
this.user = user;
}
/**
* Creates a new permission for the given user, table, column family and
* column qualifier.
* @param user the user
* @param table the table
* @param family the family, can be null if action is allowed over the entire
* table
* @param qualifier the column qualifier, can be null if action is allowed
* over the entire column family
* @param assigned the list of allowed actions
*/
public UserPermission(byte[] user, byte[] table, byte[] family,
byte[] qualifier, Action... assigned) {
super(table, family, qualifier, assigned);
this.user = user;
}
/**
* Creates a new instance for the given user, table, column family and
* qualifier, matching the actions with the given codes.
* @param user the user
* @param table the table
* @param family the family, can be null if action is allowed over the entire
* table
* @param qualifier the column qualifier, can be null if action is allowed
* over the entire column family
* @param actionCodes the list of allowed action codes
*/
public UserPermission(byte[] user, byte[] table, byte[] family,
byte[] qualifier, byte[] actionCodes) {
super(table, family, qualifier, actionCodes);
this.user = user;
}
public byte[] getUser() {
return user;
}
@Override
public boolean equals(Object obj) {
if (!(obj instanceof UserPermission)) {
return false;
}
UserPermission other = (UserPermission)obj;
if ((Bytes.equals(user, other.getUser()) &&
super.equals(obj))) {
return true;
} else {
return false;
}
}
@Override
public int hashCode() {
final int prime = 37;
int result = super.hashCode();
if (user != null) {
result = prime * result + Bytes.hashCode(user);
}
return result;
}
public String toString() {
StringBuilder str = new StringBuilder("UserPermission: ")
.append("user=").append(Bytes.toString(user))
.append(", ").append(super.toString());
return str.toString();
}
@Override
public void readFields(DataInput in) throws IOException {
super.readFields(in);
user = Bytes.readByteArray(in);
}
@Override
public void write(DataOutput out) throws IOException {
super.write(out);
Bytes.writeByteArray(out, user);
}
}

View File

@ -0,0 +1,164 @@
/*
* 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 org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.apache.zookeeper.KeeperException;
import java.io.IOException;
import java.util.List;
/**
* Handles synchronization of access control list entries and updates
* throughout all nodes in the cluster. The {@link AccessController} instance
* on the {@code _acl_} table regions, creates a znode for each table as
* {@code /hbase/acl/tablename}, with the znode data containing a serialized
* list of the permissions granted for the table. The {@code AccessController}
* instances on all other cluster hosts watch the znodes for updates, which
* trigger updates in the {@link TableAuthManager} permission cache.
*/
public class ZKPermissionWatcher extends ZooKeeperListener {
private static Log LOG = LogFactory.getLog(ZKPermissionWatcher.class);
// parent node for permissions lists
static final String ACL_NODE = "acl";
TableAuthManager authManager;
String aclZNode;
public ZKPermissionWatcher(ZooKeeperWatcher watcher,
TableAuthManager authManager, Configuration conf) {
super(watcher);
this.authManager = authManager;
String aclZnodeParent = conf.get("zookeeper.znode.acl.parent", ACL_NODE);
this.aclZNode = ZKUtil.joinZNode(watcher.baseZNode, aclZnodeParent);
}
public void start() throws KeeperException {
watcher.registerListener(this);
if (ZKUtil.watchAndCheckExists(watcher, aclZNode)) {
List<ZKUtil.NodeAndData> existing =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
if (existing != null) {
refreshNodes(existing);
}
}
}
@Override
public void nodeCreated(String path) {
if (path.equals(aclZNode)) {
try {
List<ZKUtil.NodeAndData> nodes =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
refreshNodes(nodes);
} catch (KeeperException ke) {
LOG.error("Error reading data from zookeeper", ke);
// only option is to abort
watcher.abort("Zookeeper error obtaining acl node children", ke);
}
}
}
@Override
public void nodeDeleted(String path) {
if (aclZNode.equals(ZKUtil.getParent(path))) {
String table = ZKUtil.getNodeName(path);
authManager.remove(Bytes.toBytes(table));
}
}
@Override
public void nodeDataChanged(String path) {
if (aclZNode.equals(ZKUtil.getParent(path))) {
// update cache on an existing table node
String table = ZKUtil.getNodeName(path);
try {
byte[] data = ZKUtil.getDataAndWatch(watcher, path);
authManager.refreshCacheFromWritable(Bytes.toBytes(table), data);
} catch (KeeperException ke) {
LOG.error("Error reading data from zookeeper for node "+table, ke);
// only option is to abort
watcher.abort("Zookeeper error getting data for node " + table, ke);
} catch (IOException ioe) {
LOG.error("Error reading permissions writables", ioe);
}
}
}
@Override
public void nodeChildrenChanged(String path) {
if (path.equals(aclZNode)) {
// table permissions changed
try {
List<ZKUtil.NodeAndData> nodes =
ZKUtil.getChildDataAndWatchForNewChildren(watcher, aclZNode);
refreshNodes(nodes);
} catch (KeeperException ke) {
LOG.error("Error reading data from zookeeper for path "+path, ke);
watcher.abort("Zookeeper error get node children for path "+path, ke);
}
}
}
private void refreshNodes(List<ZKUtil.NodeAndData> nodes) {
for (ZKUtil.NodeAndData n : nodes) {
if (n.isEmpty()) continue;
String path = n.getNode();
String table = ZKUtil.getNodeName(path);
try {
byte[] nodeData = n.getData();
if (LOG.isDebugEnabled()) {
LOG.debug("Updating permissions cache from node "+table+" with data: "+
Bytes.toStringBinary(nodeData));
}
authManager.refreshCacheFromWritable(Bytes.toBytes(table),
nodeData);
} catch (IOException ioe) {
LOG.error("Failed parsing permissions for table '" + table +
"' from zk", ioe);
}
}
}
/***
* Write a table's access controls to the permissions mirror in zookeeper
* @param tableName
* @param permsData
*/
public void writeToZookeeper(String tableName,
byte[] permsData) {
String zkNode =
ZKUtil.joinZNode(ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE),
tableName);
try {
ZKUtil.createWithParents(watcher, zkNode);
ZKUtil.updateExistingNodeData(watcher, zkNode,
permsData, -1);
} catch (KeeperException e) {
LOG.error("Failed updating permissions for table '" + tableName +
"'", e);
watcher.abort("Failed writing node "+zkNode+" to zookeeper", e);
}
}
}

View File

@ -0,0 +1,41 @@
/*
* 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 java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ipc.SecureRpcEngine;
import org.apache.hadoop.hbase.security.User;
/**
* Utility methods for testing security
*/
public class SecureTestUtil {
public static void enableSecurity(Configuration conf) throws IOException {
conf.set("hadoop.security.authorization", "false");
conf.set("hadoop.security.authentication", "simple");
conf.set("hbase.rpc.engine", SecureRpcEngine.class.getName());
conf.set("hbase.coprocessor.master.classes", AccessController.class.getName());
conf.set("hbase.coprocessor.region.classes", AccessController.class.getName());
// add the process running user to superusers
String currentUser = User.getCurrent().getName();
conf.set("hbase.superuser", "admin,"+currentUser);
}
}

View File

@ -0,0 +1,170 @@
/*
* 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.junit.Assert.*;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class TestAccessControlFilter {
private static Log LOG = LogFactory.getLog(TestAccessControlFilter.class);
private static HBaseTestingUtility TEST_UTIL;
private static User ADMIN;
private static User READER;
private static User LIMITED;
private static User DENIED;
private static byte[] TABLE = Bytes.toBytes("testtable");
private static byte[] FAMILY = Bytes.toBytes("f1");
private static byte[] PRIVATE_COL = Bytes.toBytes("private");
private static byte[] PUBLIC_COL = Bytes.toBytes("public");
@BeforeClass
public static void setupBeforeClass() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
Configuration conf = TEST_UTIL.getConfiguration();
SecureTestUtil.enableSecurity(conf);
String baseuser = User.getCurrent().getShortName();
conf.set("hbase.superuser", conf.get("hbase.superuser", "") +
String.format(",%s.hfs.0,%s.hfs.1,%s.hfs.2", baseuser, baseuser, baseuser));
TEST_UTIL.startMiniCluster();
TEST_UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME, 5000);
ADMIN = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
READER = User.createUserForTesting(conf, "reader", new String[0]);
LIMITED = User.createUserForTesting(conf, "limited", new String[0]);
DENIED = User.createUserForTesting(conf, "denied", new String[0]);
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
@Test
public void testQualifierAccess() throws Exception {
final HTable table = TEST_UTIL.createTable(TABLE, FAMILY);
// set permissions
ADMIN.runAs(new PrivilegedExceptionAction<Object>() {
@Override
public Object run() throws Exception {
HTable aclmeta = new HTable(TEST_UTIL.getConfiguration(),
AccessControlLists.ACL_TABLE_NAME);
AccessControllerProtocol acls = aclmeta.coprocessorProxy(
AccessControllerProtocol.class, Bytes.toBytes("testtable"));
TablePermission perm = new TablePermission(TABLE, null, Permission.Action.READ);
acls.grant(Bytes.toBytes(READER.getShortName()), perm);
perm = new TablePermission(TABLE, FAMILY, PUBLIC_COL, Permission.Action.READ);
acls.grant(Bytes.toBytes(LIMITED.getShortName()), perm);
return null;
}
});
// put some test data
List<Put> puts = new ArrayList<Put>(100);
for (int i=0; i<100; i++) {
Put p = new Put(Bytes.toBytes(i));
p.add(FAMILY, PRIVATE_COL, Bytes.toBytes("secret "+i));
p.add(FAMILY, PUBLIC_COL, Bytes.toBytes("info "+i));
puts.add(p);
}
table.put(puts);
// test read
READER.runAs(new PrivilegedExceptionAction<Object>() {
public Object run() throws Exception {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
// force a new RS connection
conf.set("testkey", UUID.randomUUID().toString());
HTable t = new HTable(conf, TABLE);
ResultScanner rs = t.getScanner(new Scan());
int rowcnt = 0;
for (Result r : rs) {
rowcnt++;
int rownum = Bytes.toInt(r.getRow());
assertTrue(r.containsColumn(FAMILY, PRIVATE_COL));
assertEquals("secret "+rownum, Bytes.toString(r.getValue(FAMILY, PRIVATE_COL)));
assertTrue(r.containsColumn(FAMILY, PUBLIC_COL));
assertEquals("info "+rownum, Bytes.toString(r.getValue(FAMILY, PUBLIC_COL)));
}
assertEquals("Expected 100 rows returned", 100, rowcnt);
return null;
}
});
// test read with qualifier filter
LIMITED.runAs(new PrivilegedExceptionAction<Object>() {
public Object run() throws Exception {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
// force a new RS connection
conf.set("testkey", UUID.randomUUID().toString());
HTable t = new HTable(conf, TABLE);
ResultScanner rs = t.getScanner(new Scan());
int rowcnt = 0;
for (Result r : rs) {
rowcnt++;
int rownum = Bytes.toInt(r.getRow());
assertFalse(r.containsColumn(FAMILY, PRIVATE_COL));
assertTrue(r.containsColumn(FAMILY, PUBLIC_COL));
assertEquals("info " + rownum, Bytes.toString(r.getValue(FAMILY, PUBLIC_COL)));
}
assertEquals("Expected 100 rows returned", 100, rowcnt);
return null;
}
});
// test as user with no permission
DENIED.runAs(new PrivilegedExceptionAction(){
public Object run() throws Exception {
try {
Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
// force a new RS connection
conf.set("testkey", UUID.randomUUID().toString());
HTable t = new HTable(conf, TABLE);
ResultScanner rs = t.getScanner(new Scan());
fail("Attempt to open scanner should have been denied");
} catch (AccessDeniedException ade) {
// expected
}
return null;
}
});
}
}

View File

@ -0,0 +1,980 @@
/*
* 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 java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HServerAddress;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.*;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import static org.junit.Assert.*;
/**
* Performs authorization checks for common operations, according to different
* levels of authorized users.
*/
public class TestAccessController {
private static Log LOG = LogFactory.getLog(TestAccessController.class);
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static Configuration conf;
// user with all permissions
private static User SUPERUSER;
// table owner user
private static User USER_OWNER;
// user with rw permissions
private static User USER_RW;
// user with read-only permissions
private static User USER_RO;
// user with no permissions
private static User USER_NONE;
private static byte[] TEST_TABLE = Bytes.toBytes("testtable");
private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
private static MasterCoprocessorEnvironment CP_ENV;
private static AccessController ACCESS_CONTROLLER;
@BeforeClass
public static void setupBeforeClass() throws Exception {
// setup configuration
conf = TEST_UTIL.getConfiguration();
SecureTestUtil.enableSecurity(conf);
TEST_UTIL.startMiniCluster();
MasterCoprocessorHost cpHost = TEST_UTIL.getMiniHBaseCluster()
.getMaster().getCoprocessorHost();
cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
ACCESS_CONTROLLER = (AccessController)cpHost.findCoprocessor(
AccessController.class.getName());
CP_ENV = cpHost.createEnvironment(AccessController.class, ACCESS_CONTROLLER,
Coprocessor.PRIORITY_HIGHEST, 1, conf);
// create a set of test users
SUPERUSER = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
USER_OWNER = User.createUserForTesting(conf, "owner", new String[0]);
USER_RW = User.createUserForTesting(conf, "rwuser", new String[0]);
USER_RO = User.createUserForTesting(conf, "rouser", new String[0]);
USER_NONE = User.createUserForTesting(conf, "nouser", new String[0]);
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
htd.setOwnerString(USER_OWNER.getShortName());
admin.createTable(htd);
// initilize access control
HTable meta = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
AccessControllerProtocol protocol =
meta.coprocessorProxy(AccessControllerProtocol.class, TEST_TABLE);
protocol.grant(Bytes.toBytes(USER_RW.getShortName()),
new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ,
Permission.Action.WRITE));
protocol.grant(Bytes.toBytes(USER_RO.getShortName()),
new TablePermission(TEST_TABLE, TEST_FAMILY, Permission.Action.READ));
}
@AfterClass
public static void tearDownAfterClass() throws Exception {
TEST_UTIL.shutdownMiniCluster();
}
public void verifyAllowed(User user, PrivilegedExceptionAction action)
throws Exception {
try {
user.runAs(action);
} catch (AccessDeniedException ade) {
fail("Expected action to pass for user '" + user.getShortName() +
"' but was denied");
}
}
public void verifyDenied(User user, PrivilegedExceptionAction action)
throws Exception {
try {
user.runAs(action);
fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
} catch (RetriesExhaustedWithDetailsException e) {
// in case of batch operations, and put, the client assembles a
// RetriesExhaustedWithDetailsException instead of throwing an
// AccessDeniedException
boolean isAccessDeniedException = false;
for ( Throwable ex : e.getCauses()) {
if (ex instanceof AccessDeniedException) {
isAccessDeniedException = true;
break;
}
}
if (!isAccessDeniedException ) {
fail("Not receiving AccessDeniedException for user '" +
user.getShortName() + "'");
}
} catch (AccessDeniedException ade) {
// expected result
}
}
@Test
public void testTableCreate() throws Exception {
PrivilegedExceptionAction createTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
HTableDescriptor htd = new HTableDescriptor("testnewtable");
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
ACCESS_CONTROLLER.preCreateTable(
ObserverContext.createAndPrepare(CP_ENV, null), htd, null);
return null;
}
};
// verify that superuser can create tables
verifyAllowed(SUPERUSER, createTable);
// all others should be denied
verifyDenied(USER_OWNER, createTable);
verifyDenied(USER_RW, createTable);
verifyDenied(USER_RO, createTable);
verifyDenied(USER_NONE, createTable);
}
@Test
public void testTableModify() throws Exception {
PrivilegedExceptionAction disableTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
HTableDescriptor htd = new HTableDescriptor(TEST_TABLE);
htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
htd.addFamily(new HColumnDescriptor("fam_"+User.getCurrent().getShortName()));
ACCESS_CONTROLLER.preModifyTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, htd);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, disableTable);
verifyDenied(USER_RW, disableTable);
verifyDenied(USER_RO, disableTable);
verifyDenied(USER_NONE, disableTable);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, disableTable);
}
@Test
public void testTableDelete() throws Exception {
PrivilegedExceptionAction disableTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preDeleteTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, disableTable);
verifyDenied(USER_RW, disableTable);
verifyDenied(USER_RO, disableTable);
verifyDenied(USER_NONE, disableTable);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, disableTable);
}
@Test
public void testAddColumn() throws Exception {
final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preAddColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, hcd);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testModifyColumn() throws Exception {
final HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY);
hcd.setMaxVersions(10);
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preModifyColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, hcd);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testDeleteColumn() throws Exception {
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preDeleteColumn(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE, TEST_FAMILY);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testTableDisable() throws Exception {
PrivilegedExceptionAction disableTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, disableTable);
verifyDenied(USER_RW, disableTable);
verifyDenied(USER_RO, disableTable);
verifyDenied(USER_NONE, disableTable);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, disableTable);
}
@Test
public void testTableEnable() throws Exception {
PrivilegedExceptionAction enableTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preEnableTable(ObserverContext.createAndPrepare(CP_ENV, null), TEST_TABLE);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, enableTable);
verifyDenied(USER_RW, enableTable);
verifyDenied(USER_RO, enableTable);
verifyDenied(USER_NONE, enableTable);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, enableTable);
}
@Test
public void testMove() throws Exception {
HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
regions.entrySet().iterator().next();
final ServerName server = TEST_UTIL.getHBaseCluster().getRegionServer(0).getServerName();
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preMove(ObserverContext.createAndPrepare(CP_ENV, null),
firstRegion.getKey(), server, server);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testAssign() throws Exception {
HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
regions.entrySet().iterator().next();
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preAssign(ObserverContext.createAndPrepare(CP_ENV, null),
firstRegion.getKey());
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testUnassign() throws Exception {
HTable table = new HTable(TEST_UTIL.getConfiguration(), TEST_TABLE);
Map<HRegionInfo,HServerAddress> regions = table.getRegionsInfo();
final Map.Entry<HRegionInfo,HServerAddress> firstRegion =
regions.entrySet().iterator().next();
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preUnassign(ObserverContext.createAndPrepare(CP_ENV, null),
firstRegion.getKey(), false);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testBalance() throws Exception {
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preBalance(ObserverContext.createAndPrepare(CP_ENV, null));
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testBalanceSwitch() throws Exception {
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preBalanceSwitch(ObserverContext.createAndPrepare(CP_ENV, null), true);
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testShutdown() throws Exception {
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preShutdown(ObserverContext.createAndPrepare(CP_ENV, null));
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
@Test
public void testStopMaster() throws Exception {
PrivilegedExceptionAction action = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preStopMaster(ObserverContext.createAndPrepare(CP_ENV, null));
return null;
}
};
// all others should be denied
verifyDenied(USER_OWNER, action);
verifyDenied(USER_RW, action);
verifyDenied(USER_RO, action);
verifyDenied(USER_NONE, action);
// verify that superuser can create tables
verifyAllowed(SUPERUSER, action);
}
private void verifyWrite(PrivilegedExceptionAction action) throws Exception {
// should be denied
verifyDenied(USER_NONE, action);
verifyDenied(USER_RO, action);
// should be allowed
verifyAllowed(SUPERUSER, action);
verifyAllowed(USER_OWNER, action);
verifyAllowed(USER_RW, action);
}
private void verifyRead(PrivilegedExceptionAction action) throws Exception {
// should be denied
verifyDenied(USER_NONE, action);
// should be allowed
verifyAllowed(SUPERUSER, action);
verifyAllowed(USER_OWNER, action);
verifyAllowed(USER_RW, action);
verifyAllowed(USER_RO, action);
}
@Test
public void testRead() throws Exception {
// get action
PrivilegedExceptionAction getAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Get g = new Get(Bytes.toBytes("random_row"));
g.addFamily(TEST_FAMILY);
HTable t = new HTable(conf, TEST_TABLE);
t.get(g);
return null;
}
};
verifyRead(getAction);
// action for scanning
PrivilegedExceptionAction scanAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Scan s = new Scan();
s.addFamily(TEST_FAMILY);
HTable table = new HTable(conf, TEST_TABLE);
ResultScanner scanner = table.getScanner(s);
try {
for (Result r = scanner.next(); r != null; r = scanner.next()) {
// do nothing
}
} catch (IOException e) {
} finally {
scanner.close();
}
return null;
}
};
verifyRead(scanAction);
}
@Test
// test put, delete, increment
public void testWrite() throws Exception {
// put action
PrivilegedExceptionAction putAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Put p = new Put(Bytes.toBytes("random_row"));
p.add(TEST_FAMILY, Bytes.toBytes("Qualifier"), Bytes.toBytes(1));
HTable t = new HTable(conf, TEST_TABLE);
t.put(p);
return null;
}
};
verifyWrite(putAction);
// delete action
PrivilegedExceptionAction deleteAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Delete d = new Delete(Bytes.toBytes("random_row"));
d.deleteFamily(TEST_FAMILY);
HTable t = new HTable(conf, TEST_TABLE);
t.delete(d);
return null;
}
};
verifyWrite(deleteAction);
// increment action
PrivilegedExceptionAction incrementAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Increment inc = new Increment(Bytes.toBytes("random_row"));
inc.addColumn(TEST_FAMILY, Bytes.toBytes("Qualifier"), 1);
HTable t = new HTable(conf, TEST_TABLE);
t.increment(inc);
return null;
}
};
verifyWrite(incrementAction);
}
@Test
public void testGrantRevoke() throws Exception {
final byte[] tableName = Bytes.toBytes("TempTable");
final byte[] family1 = Bytes.toBytes("f1");
final byte[] family2 = Bytes.toBytes("f2");
final byte[] qualifier = Bytes.toBytes("q");
// create table
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(tableName)) {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(family1));
htd.addFamily(new HColumnDescriptor(family2));
htd.setOwnerString(USER_OWNER.getShortName());
admin.createTable(htd);
// create temp users
User user = User.createUserForTesting(TEST_UTIL.getConfiguration(),
"user", new String[0]);
// perms only stored against the first region
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
AccessControllerProtocol protocol =
acl.coprocessorProxy(AccessControllerProtocol.class,
tableName);
// prepare actions:
PrivilegedExceptionAction putActionAll = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Put p = new Put(Bytes.toBytes("a"));
p.add(family1, qualifier, Bytes.toBytes("v1"));
p.add(family2, qualifier, Bytes.toBytes("v2"));
HTable t = new HTable(conf, tableName);
t.put(p);
return null;
}
};
PrivilegedExceptionAction putAction1 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Put p = new Put(Bytes.toBytes("a"));
p.add(family1, qualifier, Bytes.toBytes("v1"));
HTable t = new HTable(conf, tableName);
t.put(p);
return null;
}
};
PrivilegedExceptionAction putAction2 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Put p = new Put(Bytes.toBytes("a"));
p.add(family2, qualifier, Bytes.toBytes("v2"));
HTable t = new HTable(conf, tableName);
t.put(p);
return null;
}
};
PrivilegedExceptionAction getActionAll = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Get g = new Get(Bytes.toBytes("random_row"));
g.addFamily(family1);
g.addFamily(family2);
HTable t = new HTable(conf, tableName);
t.get(g);
return null;
}
};
PrivilegedExceptionAction getAction1 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Get g = new Get(Bytes.toBytes("random_row"));
g.addFamily(family1);
HTable t = new HTable(conf, tableName);
t.get(g);
return null;
}
};
PrivilegedExceptionAction getAction2 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Get g = new Get(Bytes.toBytes("random_row"));
g.addFamily(family2);
HTable t = new HTable(conf, tableName);
t.get(g);
return null;
}
};
PrivilegedExceptionAction deleteActionAll = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Delete d = new Delete(Bytes.toBytes("random_row"));
d.deleteFamily(family1);
d.deleteFamily(family2);
HTable t = new HTable(conf, tableName);
t.delete(d);
return null;
}
};
PrivilegedExceptionAction deleteAction1 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Delete d = new Delete(Bytes.toBytes("random_row"));
d.deleteFamily(family1);
HTable t = new HTable(conf, tableName);
t.delete(d);
return null;
}
};
PrivilegedExceptionAction deleteAction2 = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Delete d = new Delete(Bytes.toBytes("random_row"));
d.deleteFamily(family2);
HTable t = new HTable(conf, tableName);
t.delete(d);
return null;
}
};
// initial check:
verifyDenied(user, getActionAll);
verifyDenied(user, getAction1);
verifyDenied(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyDenied(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyDenied(user, deleteAction2);
// grant table read permission
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, null, Permission.Action.READ));
Thread.sleep(100);
// check
verifyAllowed(user, getActionAll);
verifyAllowed(user, getAction1);
verifyAllowed(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyDenied(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyDenied(user, deleteAction2);
// grant table write permission
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, null, Permission.Action.WRITE));
Thread.sleep(100);
verifyDenied(user, getActionAll);
verifyDenied(user, getAction1);
verifyDenied(user, getAction2);
verifyAllowed(user, putActionAll);
verifyAllowed(user, putAction1);
verifyAllowed(user, putAction2);
verifyAllowed(user, deleteActionAll);
verifyAllowed(user, deleteAction1);
verifyAllowed(user, deleteAction2);
// revoke table permission
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, null, Permission.Action.READ,
Permission.Action.WRITE));
protocol.revoke(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, null));
Thread.sleep(100);
verifyDenied(user, getActionAll);
verifyDenied(user, getAction1);
verifyDenied(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyDenied(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyDenied(user, deleteAction2);
// grant column family read permission
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1, Permission.Action.READ));
Thread.sleep(100);
verifyAllowed(user, getActionAll);
verifyAllowed(user, getAction1);
verifyDenied(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyDenied(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyDenied(user, deleteAction2);
// grant column family write permission
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family2, Permission.Action.WRITE));
Thread.sleep(100);
verifyAllowed(user, getActionAll);
verifyAllowed(user, getAction1);
verifyDenied(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyAllowed(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyAllowed(user, deleteAction2);
// revoke column family permission
protocol.revoke(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family2));
Thread.sleep(100);
verifyAllowed(user, getActionAll);
verifyAllowed(user, getAction1);
verifyDenied(user, getAction2);
verifyDenied(user, putActionAll);
verifyDenied(user, putAction1);
verifyDenied(user, putAction2);
verifyDenied(user, deleteActionAll);
verifyDenied(user, deleteAction1);
verifyDenied(user, deleteAction2);
// delete table
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
private boolean hasFoundUserPermission(UserPermission userPermission,
List<UserPermission> perms) {
return perms.contains(userPermission);
}
@Test
public void testGrantRevokeAtQualifierLevel() throws Exception {
final byte[] tableName = Bytes.toBytes("testGrantRevokeAtQualifierLevel");
final byte[] family1 = Bytes.toBytes("f1");
final byte[] family2 = Bytes.toBytes("f2");
final byte[] qualifier = Bytes.toBytes("q");
// create table
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(tableName)) {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(family1));
htd.addFamily(new HColumnDescriptor(family2));
htd.setOwnerString(USER_OWNER.getShortName());
admin.createTable(htd);
// create temp users
User user = User.createUserForTesting(TEST_UTIL.getConfiguration(),
"user", new String[0]);
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
AccessControllerProtocol protocol =
acl.coprocessorProxy(AccessControllerProtocol.class, tableName);
PrivilegedExceptionAction getQualifierAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Get g = new Get(Bytes.toBytes("random_row"));
g.addColumn(family1, qualifier);
HTable t = new HTable(conf, tableName);
t.get(g);
return null;
}
};
PrivilegedExceptionAction putQualifierAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Put p = new Put(Bytes.toBytes("random_row"));
p.add(family1, qualifier, Bytes.toBytes("v1"));
HTable t = new HTable(conf, tableName);
t.put(p);
return null;
}
};
PrivilegedExceptionAction deleteQualifierAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
Delete d = new Delete(Bytes.toBytes("random_row"));
d.deleteColumn(family1, qualifier);
//d.deleteFamily(family1);
HTable t = new HTable(conf, tableName);
t.delete(d);
return null;
}
};
protocol.revoke(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1));
verifyDenied(user, getQualifierAction);
verifyDenied(user, putQualifierAction);
verifyDenied(user, deleteQualifierAction);
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1, qualifier,
Permission.Action.READ));
Thread.sleep(100);
verifyAllowed(user, getQualifierAction);
verifyDenied(user, putQualifierAction);
verifyDenied(user, deleteQualifierAction);
// only grant write permission
// TODO: comment this portion after HBASE-3583
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1, qualifier,
Permission.Action.WRITE));
Thread.sleep(100);
verifyDenied(user, getQualifierAction);
verifyAllowed(user, putQualifierAction);
verifyAllowed(user, deleteQualifierAction);
// grant both read and write permission.
protocol.grant(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1, qualifier,
Permission.Action.READ, Permission.Action.WRITE));
Thread.sleep(100);
verifyAllowed(user, getQualifierAction);
verifyAllowed(user, putQualifierAction);
verifyAllowed(user, deleteQualifierAction);
// revoke family level permission won't impact column level.
protocol.revoke(Bytes.toBytes(user.getShortName()),
new TablePermission(tableName, family1, qualifier));
Thread.sleep(100);
verifyDenied(user, getQualifierAction);
verifyDenied(user, putQualifierAction);
verifyDenied(user, deleteQualifierAction);
// delete table
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
@Test
public void testPermissionList() throws Exception {
final byte[] tableName = Bytes.toBytes("testPermissionList");
final byte[] family1 = Bytes.toBytes("f1");
final byte[] family2 = Bytes.toBytes("f2");
final byte[] qualifier = Bytes.toBytes("q");
final byte[] user = Bytes.toBytes("user");
// create table
HBaseAdmin admin = TEST_UTIL.getHBaseAdmin();
if (admin.tableExists(tableName)) {
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
HTableDescriptor htd = new HTableDescriptor(tableName);
htd.addFamily(new HColumnDescriptor(family1));
htd.addFamily(new HColumnDescriptor(family2));
htd.setOwnerString(USER_OWNER.getShortName());
admin.createTable(htd);
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
AccessControllerProtocol protocol =
acl.coprocessorProxy(AccessControllerProtocol.class, tableName);
List<UserPermission> perms = protocol.getUserPermissions(tableName);
UserPermission up = new UserPermission(user,
tableName, family1, qualifier, Permission.Action.READ);
assertFalse("User should not be granted permission: " + up.toString(),
hasFoundUserPermission(up, perms));
// grant read permission
UserPermission upToSet = new UserPermission(user,
tableName, family1, qualifier, Permission.Action.READ);
protocol.grant(user, upToSet);
perms = protocol.getUserPermissions(tableName);
UserPermission upToVerify = new UserPermission(user,
tableName, family1, qualifier, Permission.Action.READ);
assertTrue("User should be granted permission: " + upToVerify.toString(),
hasFoundUserPermission(upToVerify, perms));
upToVerify = new UserPermission(user, tableName, family1, qualifier,
Permission.Action.WRITE);
assertFalse("User should not be granted permission: " + upToVerify.toString(),
hasFoundUserPermission(upToVerify, perms));
// grant read+write
upToSet = new UserPermission(user, tableName, family1, qualifier,
Permission.Action.WRITE, Permission.Action.READ);
protocol.grant(user, upToSet);
perms = protocol.getUserPermissions(tableName);
upToVerify = new UserPermission(user, tableName, family1, qualifier,
Permission.Action.WRITE, Permission.Action.READ);
assertTrue("User should be granted permission: " + upToVerify.toString(),
hasFoundUserPermission(upToVerify, perms));
protocol.revoke(user, upToSet);
perms = protocol.getUserPermissions(tableName);
assertFalse("User should not be granted permission: " + upToVerify.toString(),
hasFoundUserPermission(upToVerify, perms));
// delete table
admin.disableTable(tableName);
admin.deleteTable(tableName);
}
}

View File

@ -0,0 +1,305 @@
/*
* 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 com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.client.HBaseAdmin;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.*;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import static org.junit.Assert.*;
/**
* Test the reading and writing of access permissions on {@code _acl_} table.
*/
public class TestTablePermissions {
private static final Log LOG = LogFactory.getLog(TestTablePermissions.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static ZooKeeperWatcher ZKW;
private final static Abortable ABORTABLE = new Abortable() {
private final AtomicBoolean abort = new AtomicBoolean(false);
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
abort.set(true);
}
@Override
public boolean isAborted() {
return abort.get();
}
};
private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
private static byte[] TEST_TABLE2 = Bytes.toBytes("perms_test2");
private static byte[] TEST_FAMILY = Bytes.toBytes("f1");
private static byte[] TEST_QUALIFIER = Bytes.toBytes("col1");
@BeforeClass
public static void beforeClass() throws Exception {
// setup configuration
Configuration conf = UTIL.getConfiguration();
SecureTestUtil.enableSecurity(conf);
UTIL.startMiniCluster();
ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
"TestTablePermissions", ABORTABLE);
UTIL.createTable(TEST_TABLE, TEST_FAMILY);
UTIL.createTable(TEST_TABLE2, TEST_FAMILY);
}
@AfterClass
public static void afterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Test
public void testBasicWrite() throws Exception {
Configuration conf = UTIL.getConfiguration();
// add some permissions
AccessControlLists.addTablePermission(conf, TEST_TABLE,
"george", new TablePermission(TEST_TABLE, null,
TablePermission.Action.READ, TablePermission.Action.WRITE));
AccessControlLists.addTablePermission(conf, TEST_TABLE,
"hubert", new TablePermission(TEST_TABLE, null,
TablePermission.Action.READ));
AccessControlLists.addTablePermission(conf, TEST_TABLE,
"humphrey", new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
TablePermission.Action.READ));
// retrieve the same
ListMultimap<String,TablePermission> perms =
AccessControlLists.getTablePermissions(conf, TEST_TABLE);
List<TablePermission> userPerms = perms.get("george");
assertNotNull("Should have permissions for george", userPerms);
assertEquals("Should have 1 permission for george", 1, userPerms.size());
TablePermission permission = userPerms.get(0);
assertTrue("Permission should be for " + TEST_TABLE,
Bytes.equals(TEST_TABLE, permission.getTable()));
assertNull("Column family should be empty", permission.getFamily());
// check actions
assertNotNull(permission.getActions());
assertEquals(2, permission.getActions().length);
List<TablePermission.Action> actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ));
assertTrue(actions.contains(TablePermission.Action.WRITE));
userPerms = perms.get("hubert");
assertNotNull("Should have permissions for hubert", userPerms);
assertEquals("Should have 1 permission for hubert", 1, userPerms.size());
permission = userPerms.get(0);
assertTrue("Permission should be for " + TEST_TABLE,
Bytes.equals(TEST_TABLE, permission.getTable()));
assertNull("Column family should be empty", permission.getFamily());
// check actions
assertNotNull(permission.getActions());
assertEquals(1, permission.getActions().length);
actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ));
assertFalse(actions.contains(TablePermission.Action.WRITE));
userPerms = perms.get("humphrey");
assertNotNull("Should have permissions for humphrey", userPerms);
assertEquals("Should have 1 permission for humphrey", 1, userPerms.size());
permission = userPerms.get(0);
assertTrue("Permission should be for " + TEST_TABLE,
Bytes.equals(TEST_TABLE, permission.getTable()));
assertTrue("Permission should be for family " + TEST_FAMILY,
Bytes.equals(TEST_FAMILY, permission.getFamily()));
assertTrue("Permission should be for qualifier " + TEST_QUALIFIER,
Bytes.equals(TEST_QUALIFIER, permission.getQualifier()));
// check actions
assertNotNull(permission.getActions());
assertEquals(1, permission.getActions().length);
actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ));
assertFalse(actions.contains(TablePermission.Action.WRITE));
// table 2 permissions
AccessControlLists.addTablePermission(conf, TEST_TABLE2, "hubert",
new TablePermission(TEST_TABLE2, null,
TablePermission.Action.READ, TablePermission.Action.WRITE));
// check full load
Map<byte[],ListMultimap<String,TablePermission>> allPerms =
AccessControlLists.loadAll(conf);
assertEquals("Full permission map should have entries for both test tables",
2, allPerms.size());
userPerms = allPerms.get(TEST_TABLE).get("hubert");
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
assertTrue(Bytes.equals(TEST_TABLE, permission.getTable()));
assertEquals(1, permission.getActions().length);
assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
userPerms = allPerms.get(TEST_TABLE2).get("hubert");
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
assertTrue(Bytes.equals(TEST_TABLE2, permission.getTable()));
assertEquals(2, permission.getActions().length);
actions = Arrays.asList(permission.getActions());
assertTrue(actions.contains(TablePermission.Action.READ));
assertTrue(actions.contains(TablePermission.Action.WRITE));
}
@Test
public void testPersistence() throws Exception {
Configuration conf = UTIL.getConfiguration();
AccessControlLists.addTablePermission(conf, TEST_TABLE, "albert",
new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
AccessControlLists.addTablePermission(conf, TEST_TABLE, "betty",
new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
TablePermission.Action.WRITE));
AccessControlLists.addTablePermission(conf, TEST_TABLE, "clark",
new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ));
AccessControlLists.addTablePermission(conf, TEST_TABLE, "dwight",
new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER,
TablePermission.Action.WRITE));
// verify permissions survive changes in table metadata
ListMultimap<String,TablePermission> preperms =
AccessControlLists.getTablePermissions(conf, TEST_TABLE);
HTable table = new HTable(conf, TEST_TABLE);
table.put(new Put(Bytes.toBytes("row1"))
.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v1")));
table.put(new Put(Bytes.toBytes("row2"))
.add(TEST_FAMILY, TEST_QUALIFIER, Bytes.toBytes("v2")));
HBaseAdmin admin = UTIL.getHBaseAdmin();
admin.split(TEST_TABLE);
// wait for split
Thread.sleep(10000);
ListMultimap<String,TablePermission> postperms =
AccessControlLists.getTablePermissions(conf, TEST_TABLE);
checkMultimapEqual(preperms, postperms);
}
@Test
public void testSerialization() throws Exception {
Configuration conf = UTIL.getConfiguration();
ListMultimap<String,TablePermission> permissions = ArrayListMultimap.create();
permissions.put("george", new TablePermission(TEST_TABLE, null,
TablePermission.Action.READ));
permissions.put("george", new TablePermission(TEST_TABLE, TEST_FAMILY,
TablePermission.Action.WRITE));
permissions.put("george", new TablePermission(TEST_TABLE2, null,
TablePermission.Action.READ));
permissions.put("hubert", new TablePermission(TEST_TABLE2, null,
TablePermission.Action.READ, TablePermission.Action.WRITE));
ByteArrayOutputStream bos = new ByteArrayOutputStream();
AccessControlLists.writePermissions(new DataOutputStream(bos),
permissions, conf);
ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray());
ListMultimap<String,TablePermission> copy =
AccessControlLists.readPermissions(new DataInputStream(bis), conf);
checkMultimapEqual(permissions, copy);
}
public void checkMultimapEqual(ListMultimap<String,TablePermission> first,
ListMultimap<String,TablePermission> second) {
assertEquals(first.size(), second.size());
for (String key : first.keySet()) {
List<TablePermission> firstPerms = first.get(key);
List<TablePermission> secondPerms = second.get(key);
assertNotNull(secondPerms);
assertEquals(firstPerms.size(), secondPerms.size());
LOG.info("First permissions: "+firstPerms.toString());
LOG.info("Second permissions: "+secondPerms.toString());
for (TablePermission p : firstPerms) {
assertTrue("Permission "+p.toString()+" not found", secondPerms.contains(p));
}
}
}
@Test
public void testEquals() throws Exception {
TablePermission p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
TablePermission p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
assertTrue(p1.equals(p2));
assertTrue(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ, TablePermission.Action.WRITE);
p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.WRITE, TablePermission.Action.READ);
assertTrue(p1.equals(p2));
assertTrue(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ, TablePermission.Action.WRITE);
p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.WRITE, TablePermission.Action.READ);
assertTrue(p1.equals(p2));
assertTrue(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.READ, TablePermission.Action.WRITE);
p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TEST_QUALIFIER, TablePermission.Action.WRITE, TablePermission.Action.READ);
assertTrue(p1.equals(p2));
assertTrue(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
p2 = new TablePermission(TEST_TABLE, TEST_FAMILY, TablePermission.Action.READ);
assertFalse(p1.equals(p2));
assertFalse(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.WRITE);
assertFalse(p1.equals(p2));
assertFalse(p2.equals(p1));
p2 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ, TablePermission.Action.WRITE);
assertFalse(p1.equals(p2));
assertFalse(p2.equals(p1));
p1 = new TablePermission(TEST_TABLE, null, TablePermission.Action.READ);
p2 = new TablePermission(TEST_TABLE2, null, TablePermission.Action.READ);
assertFalse(p1.equals(p2));
assertFalse(p2.equals(p1));
p2 = new TablePermission(TEST_TABLE, null);
assertFalse(p1.equals(p2));
assertFalse(p2.equals(p1));
}
}

View File

@ -0,0 +1,152 @@
/*
* 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.junit.Assert.*;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.*;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
/**
* Test the reading and writing of access permissions to and from zookeeper.
*/
public class TestZKPermissionsWatcher {
private static final Log LOG = LogFactory.getLog(TestZKPermissionsWatcher.class);
private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static TableAuthManager AUTH_A;
private static TableAuthManager AUTH_B;
private final static Abortable ABORTABLE = new Abortable() {
private final AtomicBoolean abort = new AtomicBoolean(false);
@Override
public void abort(String why, Throwable e) {
LOG.info(why, e);
abort.set(true);
}
@Override
public boolean isAborted() {
return abort.get();
}
};
private static byte[] TEST_TABLE = Bytes.toBytes("perms_test");
@BeforeClass
public static void beforeClass() throws Exception {
// setup configuration
Configuration conf = UTIL.getConfiguration();
SecureTestUtil.enableSecurity(conf);
// start minicluster
UTIL.startMiniCluster();
AUTH_A = TableAuthManager.get(new ZooKeeperWatcher(conf,
"TestZKPermissionsWatcher_1", ABORTABLE), conf);
AUTH_B = TableAuthManager.get(new ZooKeeperWatcher(conf,
"TestZKPermissionsWatcher_2", ABORTABLE), conf);
}
@AfterClass
public static void afterClass() throws Exception {
UTIL.shutdownMiniCluster();
}
@Test
public void testPermissionsWatcher() throws Exception {
assertFalse(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertFalse(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
// update ACL: george RW
List<TablePermission> acl = new ArrayList<TablePermission>();
acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ,
TablePermission.Action.WRITE));
AUTH_A.setUserPermissions("george", TEST_TABLE, acl);
Thread.sleep(100);
// check it
assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
// update ACL: hubert R
acl = new ArrayList<TablePermission>();
acl.add(new TablePermission(TEST_TABLE, null, TablePermission.Action.READ));
AUTH_B.setUserPermissions("hubert", TEST_TABLE, acl);
Thread.sleep(100);
// check it
assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertTrue(AUTH_A.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.READ));
assertTrue(AUTH_B.authorizeUser("george", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertTrue(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_A.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
assertTrue(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.READ));
assertFalse(AUTH_B.authorizeUser("hubert", TEST_TABLE, null,
TablePermission.Action.WRITE));
}
}

View File

@ -35,6 +35,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.io.hfile.Compression;
import org.apache.hadoop.hbase.regionserver.StoreFile;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.WritableComparable;
@ -81,6 +82,10 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
private static final ImmutableBytesWritable MAX_FILESIZE_KEY =
new ImmutableBytesWritable(Bytes.toBytes(MAX_FILESIZE));
public static final String OWNER = "OWNER";
public static final ImmutableBytesWritable OWNER_KEY =
new ImmutableBytesWritable(Bytes.toBytes(OWNER));
/**
* <em>INTERNAL</em> Used by rest interface to access this metadata
* attribute which denotes if the table is Read Only
@ -1064,7 +1069,7 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
public static final HTableDescriptor ROOT_TABLEDESC = new HTableDescriptor(
HConstants.ROOT_TABLE_NAME,
new HColumnDescriptor[] { new HColumnDescriptor(HConstants.CATALOG_FAMILY,
10, // Ten is arbitrary number. Keep versions to help debuggging.
10, // Ten is arbitrary number. Keep versions to help debugging.
Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
HConstants.FOREVER, StoreFile.BloomType.NONE.toString(),
HConstants.REPLICATION_SCOPE_LOCAL) });
@ -1073,8 +1078,31 @@ public class HTableDescriptor implements WritableComparable<HTableDescriptor> {
public static final HTableDescriptor META_TABLEDESC = new HTableDescriptor(
HConstants.META_TABLE_NAME, new HColumnDescriptor[] {
new HColumnDescriptor(HConstants.CATALOG_FAMILY,
10, // Ten is arbitrary number. Keep versions to help debuggging.
10, // Ten is arbitrary number. Keep versions to help debugging.
Compression.Algorithm.NONE.getName(), true, true, 8 * 1024,
HConstants.FOREVER, StoreFile.BloomType.NONE.toString(),
HConstants.REPLICATION_SCOPE_LOCAL)});
public void setOwner(User owner) {
setOwnerString(owner != null ? owner.getShortName() : null);
}
// used by admin.rb:alter(table_name,*args) to update owner.
public void setOwnerString(String ownerString) {
if (ownerString != null) {
setValue(OWNER_KEY, Bytes.toBytes(ownerString));
} else {
values.remove(OWNER_KEY);
}
}
public String getOwnerString() {
if (getValue(OWNER_KEY) != null) {
return Bytes.toString(getValue(OWNER_KEY));
}
// Note that every table should have an owner (i.e. should have OWNER_KEY set).
// .META. and -ROOT- should return system user as owner, not null (see MasterFileSystem.java:bootstrap()).
return null;
}
}

View File

@ -49,10 +49,10 @@ import java.io.IOException;
*/
public abstract class BaseRegionObserver implements RegionObserver {
@Override
public void start(CoprocessorEnvironment e) { }
public void start(CoprocessorEnvironment e) throws IOException { }
@Override
public void stop(CoprocessorEnvironment e) { }
public void stop(CoprocessorEnvironment e) throws IOException { }
@Override
public void preOpen(ObserverContext<RegionCoprocessorEnvironment> e) { }

View File

@ -381,6 +381,9 @@ public class ZKUtil {
public String toString() {
return node + " (" + RegionTransitionData.fromBytes(data) + ")";
}
public boolean isEmpty() {
return (data.length == 0);
}
}
/**

View File

@ -621,6 +621,12 @@
</description>
</property>
<property>
<name>zookeeper.znode.acl.parent</name>
<value>acl</value>
<description>Root ZNode for access control lists.</description>
</property>
<property>
<name>hbase.coprocessor.region.classes</name>
<value></value>

View File

@ -75,3 +75,4 @@ require 'hbase/hbase'
require 'hbase/admin'
require 'hbase/table'
require 'hbase/replication_admin'
require 'hbase/security'

View File

@ -289,7 +289,6 @@ module Hbase
@admin.createTable(table_description)
end
#----------------------------------------------------------------------------------------------
# Check the status of alter command (number of regions reopened)
def alter_status(table_name)
# Table name should be a string
@ -336,7 +335,17 @@ module Hbase
# No method parameter, try to use the args as a column definition
unless method = arg.delete(METHOD)
# Note that we handle owner here, and also below (see (2)) as part of the "METHOD => 'table_att'" table attributes.
# In other words, if OWNER is specified, then METHOD is set to table_att.
# alter 'tablename', {OWNER => 'username'} (that is, METHOD => 'table_att' is not specified).
if arg[OWNER]
htd.setOwnerString(arg[OWNER])
@admin.modifyTable(table_name.to_java_bytes, htd)
return
end
descriptor = hcd(arg, htd)
if arg[COMPRESSION_COMPACT]
descriptor.setValue(COMPRESSION_COMPACT, arg[COMPRESSION_COMPACT])
end
@ -376,6 +385,8 @@ module Hbase
htd.setReadOnly(JBoolean.valueOf(arg[READONLY])) if arg[READONLY]
htd.setMemStoreFlushSize(JLong.valueOf(arg[MEMSTORE_FLUSHSIZE])) if arg[MEMSTORE_FLUSHSIZE]
htd.setDeferredLogFlush(JBoolean.valueOf(arg[DEFERRED_LOG_FLUSH])) if arg[DEFERRED_LOG_FLUSH]
# (2) Here, we handle the alternate syntax of ownership setting, where method => 'table_att' is specified.
htd.setOwnerString(arg[OWNER]) if arg[OWNER]
# set a coprocessor attribute
if arg.kind_of?(Hash)

View File

@ -22,6 +22,7 @@ include Java
require 'hbase/admin'
require 'hbase/table'
require 'hbase/security'
module Hbase
class Hbase
@ -52,5 +53,8 @@ module Hbase
::Hbase::RepAdmin.new(configuration, formatter)
end
def security_admin(formatter)
::Hbase::SecurityAdmin.new(configuration, formatter)
end
end
end

View File

@ -0,0 +1,128 @@
#
# 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.
#
include Java
# Wrapper for org.apache.hadoop.hbase.client.HBaseAdmin
module Hbase
class SecurityAdmin
include HBaseConstants
def initialize(configuration, formatter)
@config = configuration
@admin = org.apache.hadoop.hbase.client.HBaseAdmin.new(configuration)
@formatter = formatter
end
#----------------------------------------------------------------------------------------------
def grant(user, permissions, table_name, family=nil, qualifier=nil)
security_available?
# Table should exist
raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
htd = @admin.getTableDescriptor(table_name.to_java_bytes)
if (family != nil)
raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
end
#TODO: need to validate user name
# invoke cp endpoint to perform access control
fambytes = family.to_java_bytes if (family != nil)
qualbytes = qualifier.to_java_bytes if (qualifier != nil)
tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, permissions.to_java_bytes)
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
protocol.grant(user.to_java_bytes, tp)
end
#----------------------------------------------------------------------------------------------
def revoke(user, table_name, family=nil, qualifier=nil)
security_available?
# Table should exist
raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
htd = @admin.getTableDescriptor(table_name.to_java_bytes)
if (family != nil)
raise(ArgumentError, "Can't find a family: #{family}") unless htd.hasFamily(family.to_java_bytes)
end
fambytes = family.to_java_bytes if (family != nil)
qualbytes = qualifier.to_java_bytes if (qualifier != nil)
tp = org.apache.hadoop.hbase.security.access.TablePermission.new(table_name.to_java_bytes, fambytes, qualbytes, "".to_java_bytes)
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
protocol.revoke(user.to_java_bytes, tp)
end
#----------------------------------------------------------------------------------------------
def user_permission(table_name)
security_available?
raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config, org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
protocol = meta_table.coprocessorProxy(org.apache.hadoop.hbase.security.access.AccessControllerProtocol.java_class,
org.apache.hadoop.hbase.HConstants::EMPTY_START_ROW)
perms = protocol.getUserPermissions(table_name.to_java_bytes)
res = {}
count = 0
perms.each do |value|
user_name = String.from_java_bytes(value.getUser)
table = (value.getTable != nil) ? org.apache.hadoop.hbase.util.Bytes::toStringBinary(value.getTable) : ''
family = (value.getFamily != nil) ? org.apache.hadoop.hbase.util.Bytes::toStringBinary(value.getFamily) : ''
qualifier = (value.getQualifier != nil) ? org.apache.hadoop.hbase.util.Bytes::toStringBinary(value.getQualifier) : ''
action = org.apache.hadoop.hbase.security.access.Permission.new value.getActions
if block_given?
yield(user_name, "#{table},#{family},#{qualifier}: #{action.to_s}")
else
res[user_name] ||= {}
res[user_name][family + ":" +qualifier] = action
end
count += 1
end
return ((block_given?) ? count : res)
end
# Does table exist?
def exists?(table_name)
@admin.tableExists(table_name)
end
# Make sure that security classes are available
def security_available?()
begin
org.apache.hadoop.hbase.security.access.AccessControllerProtocol
rescue NameError
raise(ArgumentError, "DISABLED: Security features are not available in this build of HBase")
end
end
end
end

View File

@ -87,6 +87,10 @@ module Shell
@hbase_replication_admin ||= hbase.replication_admin(formatter)
end
def hbase_security_admin
@hbase_security_admin ||= hbase.security_admin(formatter)
end
def export_commands(where)
::Shell.commands.keys.each do |cmd|
where.send :instance_eval, <<-EOF
@ -284,3 +288,14 @@ Shell.load_command_group(
]
)
Shell.load_command_group(
'security',
:full_name => 'SECURITY TOOLS',
:comment => "NOTE: Above commands are only applicable if running with the AccessController coprocessor",
:commands => %w[
grant
revoke
user_permission
]
)

View File

@ -53,6 +53,10 @@ module Shell
shell.hbase_replication_admin
end
def security_admin
shell.hbase_security_admin
end
#----------------------------------------------------------------------
def formatter

View File

@ -0,0 +1,43 @@
#
# 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.
#
module Shell
module Commands
class Grant < Command
def help
return <<-EOF
Grant users specific rights to tables.
Syntax : grant <user> <permissions> <table> <column family> <column qualifier>
permissions is either zero or more letters from the set "RWXCA".
READ('R'), WRITE('W'), EXEC('X'), CREATE('C'), ADMIN('A')
For example:
hbase> grant 'bobsmith', 'RW', 't1', 'f1', 'col1'
EOF
end
def command(user, rights, table_name, family=nil, qualifier=nil)
format_simple_command do
security_admin.grant(user, rights, table_name, family, qualifier)
end
end
end
end
end

View File

@ -0,0 +1,39 @@
#
# 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.
#
module Shell
module Commands
class Revoke < Command
def help
return <<-EOF
Revoke a user's access rights to tables.
Syntax : revoke <user> <table> <column family>
For example:
hbase> revoke 'bobsmith', 't1', 'f1'
EOF
end
def command(user, table_name, family=nil, qualifier=nil)
format_simple_command do
security_admin.revoke(user, table_name, family, qualifier)
end
end
end
end
end

View File

@ -0,0 +1,46 @@
#
# 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.
#
module Shell
module Commands
class UserPermission < Command
def help
return <<-EOF
Show all table access permissions for the particular user.
Syntax : user_permission <table>
For example:
hbase> user_permission 'table1'
EOF
end
def command(table)
#format_simple_command do
#admin.user_permission(table)
now = Time.now
formatter.header(["User", "Table,Family,Qualifier:Permission"])
count = security_admin.user_permission(table) do |user, permission|
formatter.row([ user, permission])
end
formatter.footer(now, count)
end
end
end
end