HBASE-8409 Security support for namespaces

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1513666 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2013-08-13 21:49:56 +00:00
parent 48c1e7aff3
commit ac10b3c13d
26 changed files with 4604 additions and 1214 deletions

View File

@ -1565,10 +1565,10 @@ public final class ProtobufUtil {
* @return the converted Permission
*/
public static Permission toPermission(AccessControlProtos.Permission proto) {
if (proto.hasTableName()) {
if (proto.getType() != AccessControlProtos.Permission.Type.Global) {
return toTablePermission(proto);
} else {
List<Permission.Action> actions = toPermissionActions(proto.getActionList());
List<Permission.Action> actions = toPermissionActions(proto.getGlobalPermission().getActionList());
return new Permission(actions.toArray(new Permission.Action[actions.size()]));
}
}
@ -1580,19 +1580,44 @@ public final class ProtobufUtil {
* @return the converted TablePermission
*/
public static TablePermission toTablePermission(AccessControlProtos.Permission proto) {
List<Permission.Action> actions = toPermissionActions(proto.getActionList());
if(proto.getType() == AccessControlProtos.Permission.Type.Global) {
AccessControlProtos.GlobalPermission perm = proto.getGlobalPermission();
List<Permission.Action> actions = toPermissionActions(perm.getActionList());
return new TablePermission(null, null, null,
actions.toArray(new Permission.Action[actions.size()]));
}
if(proto.getType() == AccessControlProtos.Permission.Type.Namespace) {
AccessControlProtos.NamespacePermission perm = proto.getNamespacePermission();
List<Permission.Action> actions = toPermissionActions(perm.getActionList());
if(!proto.hasNamespacePermission()) {
throw new IllegalStateException("Namespace must not be empty in NamespacePermission");
}
String namespace = perm.getNamespaceName().toStringUtf8();
return new TablePermission(namespace, actions.toArray(new Permission.Action[actions.size()]));
}
if(proto.getType() == AccessControlProtos.Permission.Type.Table) {
AccessControlProtos.TablePermission perm = proto.getTablePermission();
List<Permission.Action> actions = toPermissionActions(perm.getActionList());
byte[] qualifier = null;
byte[] family = null;
TableName table = null;
if (proto.hasTableName()) table = ProtobufUtil.toTableName(proto.getTableName());
if (proto.hasFamily()) family = proto.getFamily().toByteArray();
if (proto.hasQualifier()) qualifier = proto.getQualifier().toByteArray();
if (!perm.hasTableName()) {
throw new IllegalStateException("TableName cannot be empty");
}
table = ProtobufUtil.toTableName(perm.getTableName());
if (perm.hasFamily()) family = perm.getFamily().toByteArray();
if (perm.hasQualifier()) qualifier = perm.getQualifier().toByteArray();
return new TablePermission(table, family, qualifier,
actions.toArray(new Permission.Action[actions.size()]));
}
throw new IllegalStateException("Unrecognize Perm Type: "+proto.getType());
}
/**
* Convert a client Permission to a Permission proto
@ -1601,23 +1626,47 @@ public final class ProtobufUtil {
* @return the protobuf Permission
*/
public static AccessControlProtos.Permission toPermission(Permission perm) {
AccessControlProtos.Permission.Builder builder = AccessControlProtos.Permission.newBuilder();
AccessControlProtos.Permission.Builder ret = AccessControlProtos.Permission.newBuilder();
if (perm instanceof TablePermission) {
TablePermission tablePerm = (TablePermission)perm;
if (tablePerm.hasTable()) {
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTable()));
if(tablePerm.hasNamespace()) {
ret.setType(AccessControlProtos.Permission.Type.Namespace);
AccessControlProtos.NamespacePermission.Builder builder =
AccessControlProtos.NamespacePermission.newBuilder();
builder.setNamespaceName(ByteString.copyFromUtf8(tablePerm.getNamespace()));
for (Permission.Action a : perm.getActions()) {
builder.addAction(toPermissionAction(a));
}
ret.setNamespacePermission(builder);
} else if (tablePerm.hasTable()) {
ret.setType(AccessControlProtos.Permission.Type.Table);
AccessControlProtos.TablePermission.Builder builder =
AccessControlProtos.TablePermission.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName(tablePerm.getTable()));
if (tablePerm.hasFamily()) {
builder.setFamily(ByteString.copyFrom(tablePerm.getFamily()));
}
if (tablePerm.hasQualifier()) {
builder.setQualifier(ByteString.copyFrom(tablePerm.getQualifier()));
}
}
for (Permission.Action a : perm.getActions()) {
builder.addAction(toPermissionAction(a));
}
return builder.build();
ret.setTablePermission(builder);
}
} else {
ret.setType(AccessControlProtos.Permission.Type.Global);
AccessControlProtos.GlobalPermission.Builder builder =
AccessControlProtos.GlobalPermission.newBuilder();
for (Permission.Action a : perm.getActions()) {
builder.addAction(toPermissionAction(a));
}
ret.setGlobalPermission(builder);
}
return ret.build();
}
/**
@ -1688,24 +1737,9 @@ public final class ProtobufUtil {
* @return the protobuf UserPermission
*/
public static AccessControlProtos.UserPermission toUserPermission(UserPermission perm) {
AccessControlProtos.Permission.Builder permissionBuilder =
AccessControlProtos.Permission.newBuilder();
for (Permission.Action a : perm.getActions()) {
permissionBuilder.addAction(toPermissionAction(a));
}
if (perm.hasTable()) {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(perm.getTable()));
}
if (perm.hasFamily()) {
permissionBuilder.setFamily(ByteString.copyFrom(perm.getFamily()));
}
if (perm.hasQualifier()) {
permissionBuilder.setQualifier(ByteString.copyFrom(perm.getQualifier()));
}
return AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFrom(perm.getUser()))
.setPermission(permissionBuilder)
.setPermission(toPermission(perm))
.build();
}
@ -1716,20 +1750,8 @@ public final class ProtobufUtil {
* @return the converted UserPermission
*/
public static UserPermission toUserPermission(AccessControlProtos.UserPermission proto) {
AccessControlProtos.Permission permission = proto.getPermission();
List<Permission.Action> actions = toPermissionActions(permission.getActionList());
byte[] qualifier = null;
byte[] family = null;
TableName table = null;
if (permission.hasTableName()) table = ProtobufUtil.toTableName(permission.getTableName());
if (permission.hasFamily()) family = permission.getFamily().toByteArray();
if (permission.hasQualifier()) qualifier = permission.getQualifier().toByteArray();
return new UserPermission(proto.getUser().toByteArray(),
table, family, qualifier,
actions.toArray(new Permission.Action[actions.size()]));
toTablePermission(proto.getPermission()));
}
/**
@ -1739,26 +1761,48 @@ public final class ProtobufUtil {
* @param perm the list of user and table permissions
* @return the protobuf UserTablePermissions
*/
public static AccessControlProtos.UserTablePermissions toUserTablePermissions(
public static AccessControlProtos.UsersAndPermissions toUserTablePermissions(
ListMultimap<String, TablePermission> perm) {
AccessControlProtos.UserTablePermissions.Builder builder =
AccessControlProtos.UserTablePermissions.newBuilder();
AccessControlProtos.UsersAndPermissions.Builder builder =
AccessControlProtos.UsersAndPermissions.newBuilder();
for (Map.Entry<String, Collection<TablePermission>> entry : perm.asMap().entrySet()) {
AccessControlProtos.UserTablePermissions.UserPermissions.Builder userPermBuilder =
AccessControlProtos.UserTablePermissions.UserPermissions.newBuilder();
AccessControlProtos.UsersAndPermissions.UserPermissions.Builder userPermBuilder =
AccessControlProtos.UsersAndPermissions.UserPermissions.newBuilder();
userPermBuilder.setUser(ByteString.copyFromUtf8(entry.getKey()));
for (TablePermission tablePerm: entry.getValue()) {
userPermBuilder.addPermissions(toPermission(tablePerm));
}
builder.addPermissions(userPermBuilder.build());
builder.addUserPermissions(userPermBuilder.build());
}
return builder.build();
}
/**
* A utility used to grant a user some permissions. The permissions will
* be global if table is not specified. Otherwise, they are for those
* table/column family/qualifier only.
* A utility used to grant a user global permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
* @param protocol the AccessControlService protocol proxy
* @param userShortName the short name of the user to grant permissions
* @param actions the permissions to be granted
* @throws ServiceException
*/
public static void grant(AccessControlService.BlockingInterface protocol,
String userShortName, Permission.Action... actions) throws ServiceException {
List<AccessControlProtos.Permission.Action> permActions =
Lists.newArrayListWithCapacity(actions.length);
for (Permission.Action a : actions) {
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.GrantRequest request = RequestConverter.
buildGrantRequest(userShortName, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.grant(null, request);
}
/**
* A utility used to grant a user table permissions. The permissions will
* be for a table table/column family/qualifier.
* <p>
* It's also called by the shell, in case you want to find references.
*
@ -1785,9 +1829,55 @@ public final class ProtobufUtil {
}
/**
* A utility used to revoke a user some permissions. The permissions will
* be global if table is not specified. Otherwise, they are for those
* table/column family/qualifier only.
* A utility used to grant a user namespace permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
* @param protocol the AccessControlService protocol proxy
* @param namespace the short name of the user to grant permissions
* @param actions the permissions to be granted
* @throws ServiceException
*/
public static void grant(AccessControlService.BlockingInterface protocol,
String userShortName, String namespace,
Permission.Action... actions) throws ServiceException {
List<AccessControlProtos.Permission.Action> permActions =
Lists.newArrayListWithCapacity(actions.length);
for (Permission.Action a : actions) {
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.GrantRequest request = RequestConverter.
buildGrantRequest(userShortName, namespace, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.grant(null, request);
}
/**
* A utility used to revoke a user's global permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
* @param protocol the AccessControlService protocol proxy
* @param userShortName the short name of the user to revoke permissions
* @param actions the permissions to be revoked
* @throws ServiceException
*/
public static void revoke(AccessControlService.BlockingInterface protocol,
String userShortName, Permission.Action... actions) throws ServiceException {
List<AccessControlProtos.Permission.Action> permActions =
Lists.newArrayListWithCapacity(actions.length);
for (Permission.Action a : actions) {
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.RevokeRequest request = RequestConverter.
buildRevokeRequest(userShortName, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.revoke(null, request);
}
/**
* A utility used to revoke a user's table permissions. The permissions will
* be for a table/column family/qualifier.
* <p>
* It's also called by the shell, in case you want to find references.
*
@ -1814,7 +1904,55 @@ public final class ProtobufUtil {
}
/**
* A utility used to get user permissions.
* A utility used to revoke a user's namespace permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
* @param protocol the AccessControlService protocol proxy
* @param userShortName the short name of the user to revoke permissions
* @param namespace optional table name
* @param actions the permissions to be revoked
* @throws ServiceException
*/
public static void revoke(AccessControlService.BlockingInterface protocol,
String userShortName, String namespace,
Permission.Action... actions) throws ServiceException {
List<AccessControlProtos.Permission.Action> permActions =
Lists.newArrayListWithCapacity(actions.length);
for (Permission.Action a : actions) {
permActions.add(ProtobufUtil.toPermissionAction(a));
}
AccessControlProtos.RevokeRequest request = RequestConverter.
buildRevokeRequest(userShortName, namespace, permActions.toArray(
new AccessControlProtos.Permission.Action[actions.length]));
protocol.revoke(null, request);
}
/**
* A utility used to get user's global permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
* @param protocol the AccessControlService protocol proxy
* @throws ServiceException
*/
public static List<UserPermission> getUserPermissions(
AccessControlService.BlockingInterface protocol) throws ServiceException {
AccessControlProtos.UserPermissionsRequest.Builder builder =
AccessControlProtos.UserPermissionsRequest.newBuilder();
builder.setType(AccessControlProtos.Permission.Type.Global);
AccessControlProtos.UserPermissionsRequest request = builder.build();
AccessControlProtos.UserPermissionsResponse response =
protocol.getUserPermissions(null, request);
List<UserPermission> perms = new ArrayList<UserPermission>();
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
perms.add(ProtobufUtil.toUserPermission(perm));
}
return perms;
}
/**
* A utility used to get user table permissions.
* <p>
* It's also called by the shell, in case you want to find references.
*
@ -1830,11 +1968,12 @@ public final class ProtobufUtil {
if (t != null) {
builder.setTableName(ProtobufUtil.toProtoTableName(t));
}
builder.setType(AccessControlProtos.Permission.Type.Table);
AccessControlProtos.UserPermissionsRequest request = builder.build();
AccessControlProtos.UserPermissionsResponse response =
protocol.getUserPermissions(null, request);
List<UserPermission> perms = new ArrayList<UserPermission>();
for (AccessControlProtos.UserPermission perm: response.getPermissionList()) {
for (AccessControlProtos.UserPermission perm: response.getUserPermissionList()) {
perms.add(ProtobufUtil.toUserPermission(perm));
}
return perms;
@ -1848,12 +1987,12 @@ public final class ProtobufUtil {
* @return the converted UserPermission
*/
public static ListMultimap<String, TablePermission> toUserTablePermissions(
AccessControlProtos.UserTablePermissions proto) {
AccessControlProtos.UsersAndPermissions proto) {
ListMultimap<String, TablePermission> perms = ArrayListMultimap.create();
AccessControlProtos.UserTablePermissions.UserPermissions userPerm;
AccessControlProtos.UsersAndPermissions.UserPermissions userPerm;
for (int i = 0; i < proto.getPermissionsCount(); i++) {
userPerm = proto.getPermissions(i);
for (int i = 0; i < proto.getUserPermissionsCount(); i++) {
userPerm = proto.getUserPermissions(i);
for (int j = 0; j < userPerm.getPermissionsCount(); j++) {
TablePermission tablePerm = toTablePermission(userPerm.getPermissions(j));
perms.put(userPerm.getUser().toStringUtf8(), tablePerm);

View File

@ -1201,6 +1201,32 @@ public final class RequestConverter {
ByteString.copyFrom(regionName)).build();
}
/**
* Create a request to grant user permissions.
*
* @param username the short user name who to grant permissions
* @param actions the permissions to be granted
* @return A {@link AccessControlProtos} GrantRequest
*/
public static AccessControlProtos.GrantRequest buildGrantRequest(
String username, AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.GlobalPermission.Builder permissionBuilder =
AccessControlProtos.GlobalPermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
ret.setType(AccessControlProtos.Permission.Type.Global)
.setGlobalPermission(permissionBuilder);
return AccessControlProtos.GrantRequest.newBuilder()
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(ret)
).build();
}
/**
* Create a request to grant user permissions.
*
@ -1214,26 +1240,88 @@ public final class RequestConverter {
public static AccessControlProtos.GrantRequest buildGrantRequest(
String username, TableName tableName, byte[] family, byte[] qualifier,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder permissionBuilder =
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.TablePermission.Builder permissionBuilder =
AccessControlProtos.TablePermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
if (tableName != null) {
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (tableName == null) {
throw new NullPointerException("TableName cannot be null");
}
permissionBuilder.setTableName(ProtobufUtil.toProtoTableName(tableName));
if (family != null) {
permissionBuilder.setFamily(ByteString.copyFrom(family));
}
if (qualifier != null) {
permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
}
ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder);
return AccessControlProtos.GrantRequest.newBuilder()
.setPermission(
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(permissionBuilder.build())
.setPermission(ret)
).build();
}
/**
* Create a request to grant user permissions.
*
* @param username the short user name who to grant permissions
* @param namespace optional table name the permissions apply
* @param actions the permissions to be granted
* @return A {@link AccessControlProtos} GrantRequest
*/
public static AccessControlProtos.GrantRequest buildGrantRequest(
String username, String namespace,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.NamespacePermission.Builder permissionBuilder =
AccessControlProtos.NamespacePermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
if (namespace != null) {
permissionBuilder.setNamespaceName(ByteString.copyFromUtf8(namespace));
}
ret.setType(AccessControlProtos.Permission.Type.Namespace)
.setNamespacePermission(permissionBuilder);
return AccessControlProtos.GrantRequest.newBuilder()
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(ret)
).build();
}
/**
* Create a request to revoke user permissions.
*
* @param username the short user name whose permissions to be revoked
* @param actions the permissions to be revoked
* @return A {@link AccessControlProtos} RevokeRequest
*/
public static AccessControlProtos.RevokeRequest buildRevokeRequest(
String username, AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.GlobalPermission.Builder permissionBuilder =
AccessControlProtos.GlobalPermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
ret.setType(AccessControlProtos.Permission.Type.Global)
.setGlobalPermission(permissionBuilder);
return AccessControlProtos.RevokeRequest.newBuilder()
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(ret)
).build();
}
@ -1250,8 +1338,10 @@ public final class RequestConverter {
public static AccessControlProtos.RevokeRequest buildRevokeRequest(
String username, TableName tableName, byte[] family, byte[] qualifier,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder permissionBuilder =
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.TablePermission.Builder permissionBuilder =
AccessControlProtos.TablePermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
@ -1264,12 +1354,44 @@ public final class RequestConverter {
if (qualifier != null) {
permissionBuilder.setQualifier(ByteString.copyFrom(qualifier));
}
ret.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(permissionBuilder);
return AccessControlProtos.RevokeRequest.newBuilder()
.setPermission(
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(permissionBuilder.build())
.setPermission(ret)
).build();
}
/**
* Create a request to revoke user permissions.
*
* @param username the short user name whose permissions to be revoked
* @param namespace optional table name the permissions apply
* @param actions the permissions to be revoked
* @return A {@link AccessControlProtos} RevokeRequest
*/
public static AccessControlProtos.RevokeRequest buildRevokeRequest(
String username, String namespace,
AccessControlProtos.Permission.Action... actions) {
AccessControlProtos.Permission.Builder ret =
AccessControlProtos.Permission.newBuilder();
AccessControlProtos.NamespacePermission.Builder permissionBuilder =
AccessControlProtos.NamespacePermission.newBuilder();
for (AccessControlProtos.Permission.Action a : actions) {
permissionBuilder.addAction(a);
}
if (namespace != null) {
permissionBuilder.setNamespaceName(ByteString.copyFromUtf8(namespace));
}
ret.setType(AccessControlProtos.Permission.Type.Namespace)
.setNamespacePermission(permissionBuilder);
return AccessControlProtos.RevokeRequest.newBuilder()
.setUserPermission(
AccessControlProtos.UserPermission.newBuilder()
.setUser(ByteString.copyFromUtf8(username))
.setPermission(ret)
).build();
}

View File

@ -116,7 +116,7 @@ public final class ResponseConverter {
final List<UserPermission> permissions) {
UserPermissionsResponse.Builder builder = UserPermissionsResponse.newBuilder();
for (UserPermission perm : permissions) {
builder.addPermission(ProtobufUtil.toUserPermission(perm));
builder.addUserPermission(ProtobufUtil.toUserPermission(perm));
}
return builder.build();
}

View File

@ -41,6 +41,10 @@ public class TablePermission extends Permission {
private byte[] family;
private byte[] qualifier;
//TODO refactor this class
//we need to refacting this into three classes (Global, Table, Namespace)
private String namespace;
/** Nullary constructor for Writable, do not use */
public TablePermission() {
super();
@ -87,6 +91,62 @@ public class TablePermission extends Permission {
this.qualifier = qualifier;
}
/**
* Creates a new permission for the given namespace or table, restricted to the given
* column family and qualifer, allowing the assigned actions to be performed.
* @param namespace
* @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(String namespace, TableName table, byte[] family, byte[] qualifier,
Action... assigned) {
super(assigned);
this.namespace = namespace;
this.table = table;
this.family = family;
this.qualifier = qualifier;
}
/**
* Creates a new permission for the given namespace or table, family and column qualifier,
* allowing the actions matching the provided byte codes to be performed.
* @param namespace
* @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(String namespace, TableName table, byte[] family, byte[] qualifier,
byte[] actionCodes) {
super(actionCodes);
this.namespace = namespace;
this.table = table;
this.family = family;
this.qualifier = qualifier;
}
/**
* Creates a new permission for the given namespace,
* allowing the actions matching the provided byte codes to be performed.
* @param namespace
* @param actionCodes the list of allowed action codes
*/
public TablePermission(String namespace, byte[] actionCodes) {
super(actionCodes);
this.namespace = namespace;
}
/**
* Create a new permission for the given namespace,
* allowing the given actions.
* @param namespace
* @param assigned the list of allowed actions
*/
public TablePermission(String namespace, Action... assigned) {
super(assigned);
this.namespace = namespace;
}
public boolean hasTable() {
return table != null;
}
@ -111,6 +171,32 @@ public class TablePermission extends Permission {
return qualifier;
}
public boolean hasNamespace() {
return namespace != null;
}
public String getNamespace() {
return namespace;
}
/**
* Checks that a given table operation is authorized by this permission
* instance.
*
* @param namespace the namespace where the operation is being performed
* @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(String namespace, Action action) {
if (!this.namespace.equals(namespace)) {
return false;
}
// check actions
return super.implies(action);
}
/**
* Checks that a given table operation is authorized by this permission
* instance.
@ -234,7 +320,9 @@ public class TablePermission extends Permission {
((family == null && other.getFamily() == null) ||
Bytes.equals(family, other.getFamily())) &&
((qualifier == null && other.getQualifier() == null) ||
Bytes.equals(qualifier, other.getQualifier()))
Bytes.equals(qualifier, other.getQualifier())) &&
((namespace == null && other.getNamespace() == null) ||
namespace.equals(other.getNamespace()))
)) {
return false;
}
@ -256,15 +344,28 @@ public class TablePermission extends Permission {
if (qualifier != null) {
result = prime * result + Bytes.hashCode(qualifier);
}
if (namespace != null) {
result = prime * result + namespace.hashCode();
}
return result;
}
public String toString() {
StringBuilder str = new StringBuilder("[TablePermission: ")
.append("table=").append(table)
.append(", family=").append(Bytes.toString(family))
.append(", qualifier=").append(Bytes.toString(qualifier))
.append(", actions=");
StringBuilder str = new StringBuilder("[TablePermission: ");
if(namespace != null) {
str.append("namespace=").append(namespace)
.append(", ");
}
else if(table != null) {
str.append("table=").append(table)
.append(", family=")
.append(family == null ? null : Bytes.toString(family))
.append(", qualifier=")
.append(qualifier == null ? null : Bytes.toString(qualifier))
.append(", ");
} else {
str.append("actions=");
}
if (actions != null) {
for (int i=0; i<actions.length; i++) {
if (i > 0)
@ -291,6 +392,9 @@ public class TablePermission extends Permission {
if (in.readBoolean()) {
qualifier = Bytes.readByteArray(in);
}
if(in.readBoolean()) {
namespace = Bytes.toString(Bytes.readByteArray(in));
}
}
@Override
@ -305,5 +409,9 @@ public class TablePermission extends Permission {
if (qualifier != null) {
Bytes.writeByteArray(out, qualifier);
}
out.writeBoolean(namespace != null);
if(namespace != null) {
Bytes.writeByteArray(out, Bytes.toBytes(namespace));
}
}
}

View File

@ -62,6 +62,29 @@ public class UserPermission extends TablePermission {
this.user = user;
}
/**
* Creates a new instance for the given user.
* @param user the user
* @param namespace
* @param assigned the list of allowed actions
*/
public UserPermission(byte[] user, String namespace, Action... assigned) {
super(namespace, assigned);
this.user = user;
}
/**
* Creates a new instance for the given user,
* matching the actions with the given codes.
* @param user the user
* @param namespace
* @param actionCodes the list of allowed action codes
*/
public UserPermission(byte[] user, String namespace, byte[] actionCodes) {
super(namespace, actionCodes);
this.user = user;
}
/**
* Creates a new instance for the given user, table and column family.
* @param user the user
@ -110,6 +133,18 @@ public class UserPermission extends TablePermission {
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 perm a TablePermission
*/
public UserPermission(byte[] user, TablePermission perm) {
super(perm.getNamespace(), perm.getTable(), perm.getFamily(), perm.getQualifier(),
perm.actions);
this.user = user;
}
public byte[] getUser() {
return user;
}
@ -118,8 +153,7 @@ public class UserPermission extends TablePermission {
* Returns true if this permission describes a global user permission.
*/
public boolean isGlobal() {
TableName tableName = getTable();
return(tableName == null);
return(!hasTable() && !hasNamespace());
}
@Override

View File

@ -289,6 +289,9 @@ public final class HConstants {
/** Used by HBCK to sideline backup data */
public static final String HBCK_SIDELINEDIR_NAME = ".hbck";
/** Any artifacts left from migration can be moved here */
public static final String MIGRATION_NAME = ".migration";
/** Used to construct the name of the compaction directory during compaction */
public static final String HREGION_COMPACTIONDIR_NAME = "compaction.dir";
@ -835,7 +838,7 @@ public final class HConstants {
Collections.unmodifiableList(Arrays.asList(new String[] { HREGION_LOGDIR_NAME,
HREGION_OLDLOGDIR_NAME, CORRUPT_DIR_NAME, SPLIT_LOGDIR_NAME,
HBCK_SIDELINEDIR_NAME, HFILE_ARCHIVE_DIRECTORY, SNAPSHOT_DIR_NAME, HBASE_TEMP_DIRECTORY,
OLD_SNAPSHOT_DIR_NAME, BASE_NAMESPACE_DIR}));
OLD_SNAPSHOT_DIR_NAME, BASE_NAMESPACE_DIR, MIGRATION_NAME}));
/** Directories that are not HBase user table directories */
public static final List<String> HBASE_NON_USER_TABLE_DIRS =

View File

@ -32,51 +32,72 @@ message Permission {
CREATE = 3;
ADMIN = 4;
}
repeated Action action = 1;
optional TableName tableName = 2;
optional bytes family = 3;
optional bytes qualifier = 4;
enum Type {
Global = 1;
Namespace = 2;
Table = 3;
}
required Type type = 1;
optional GlobalPermission global_permission = 2;
optional NamespacePermission namespace_permission = 3;
optional TablePermission table_permission = 4;
}
message TablePermission {
optional TableName table_name = 1;
optional bytes family = 2;
optional bytes qualifier = 3;
repeated Permission.Action action = 4;
}
message NamespacePermission {
optional bytes namespace_name = 1;
repeated Permission.Action action = 2;
}
message GlobalPermission {
repeated Permission.Action action = 1;
}
message UserPermission {
required bytes user = 1;
required Permission permission = 2;
required Permission permission = 3;
}
/**
* Content of the /hbase/acl/<table> znode.
* Content of the /hbase/acl/<table or namespace> znode.
*/
message UserTablePermissions {
message UsersAndPermissions {
message UserPermissions {
required bytes user = 1;
repeated Permission permissions = 2;
}
repeated UserPermissions permissions = 1;
repeated UserPermissions user_permissions = 1;
}
message GrantRequest {
required UserPermission permission = 1;
required UserPermission user_permission = 1;
}
message GrantResponse {
}
message RevokeRequest {
required UserPermission permission = 1;
required UserPermission user_permission = 1;
}
message RevokeResponse {
}
message UserPermissionsRequest {
optional TableName tableName = 1;
optional Permission.Type type = 1;
optional TableName table_name = 2;
optional bytes namespace_name = 3;
}
message UserPermissionsResponse {
repeated UserPermission permission = 1;
repeated UserPermission user_permission = 1;
}
message CheckPermissionsRequest {

View File

@ -20,23 +20,38 @@
package org.apache.hadoop.hbase.migration;
import com.google.common.collect.Lists;
import com.google.common.primitives.Ints;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.catalog.MetaEditor;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
import org.apache.hadoop.hbase.regionserver.wal.HLog;
import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.util.Tool;
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
/**
@ -58,6 +73,7 @@ public class NamespaceUpgrade implements Tool {
private Path sysNsDir;
private Path defNsDir;
private Path baseDirs[];
private Path backupDir;
public NamespaceUpgrade() throws IOException {
}
@ -70,6 +86,7 @@ public class NamespaceUpgrade implements Tool {
baseDirs = new Path[]{rootDir,
new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY),
new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY)};
backupDir = new Path(rootDir, HConstants.MIGRATION_NAME);
}
@ -84,11 +101,14 @@ public class NamespaceUpgrade implements Tool {
makeNamespaceDirs();
migrateMeta();
migrateACL();
migrateTables();
migrateSnapshots();
migrateMeta();
FSUtils.setVersion(fs, rootDir);
}
@ -185,6 +205,156 @@ public class NamespaceUpgrade implements Tool {
+ oldMetaRegionDir + " to " + newMetaRegionDir);
}
}
Path oldRootDir = new Path(rootDir, "-ROOT-");
if(!fs.rename(oldRootDir, backupDir)) {
throw new IllegalStateException("Failed to old data: "+oldRootDir+" to "+backupDir);
}
}
public void migrateACL() throws IOException {
TableName oldTableName = TableName.valueOf("_acl_");
Path oldTablePath = new Path(rootDir, oldTableName.getNameAsString());
if(!fs.exists(oldTablePath)) {
return;
}
LOG.info("Migrating ACL table");
TableName newTableName = AccessControlLists.ACL_TABLE_NAME;
Path newTablePath = FSUtils.getTableDir(rootDir, newTableName);
HTableDescriptor oldDesc =
readTableDescriptor(fs, getCurrentTableInfoStatus(fs, oldTablePath));
if(FSTableDescriptors.getTableInfoPath(fs, newTablePath) == null) {
LOG.info("Creating new tableDesc for ACL");
HTableDescriptor newDesc = new HTableDescriptor(oldDesc);
newDesc.setName(newTableName);
new FSTableDescriptors(this.conf).createTableDescriptorForTableDirectory(
newTablePath, newDesc, true);
}
ServerName fakeServer = new ServerName("nsupgrade",96,123);
String metaLogName = HLogUtil.getHLogDirectoryName(fakeServer.toString());
HLog metaHLog = HLogFactory.createMetaHLog(fs, rootDir,
metaLogName, conf, null,
fakeServer.toString());
HRegion meta = HRegion.openHRegion(rootDir, HRegionInfo.FIRST_META_REGIONINFO,
HTableDescriptor.META_TABLEDESC, metaHLog, conf);
HRegion region = null;
try {
for(Path regionDir : FSUtils.getRegionDirs(fs, oldTablePath)) {
LOG.info("Migrating ACL region "+regionDir.getName());
HRegionInfo oldRegionInfo = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir);
HRegionInfo newRegionInfo =
new HRegionInfo(newTableName,
oldRegionInfo.getStartKey(),
oldRegionInfo.getEndKey(),
oldRegionInfo.isSplit(),
oldRegionInfo.getRegionId());
newRegionInfo.setOffline(oldRegionInfo.isOffline());
region =
new HRegion(
HRegionFileSystem.openRegionFromFileSystem(conf, fs, oldTablePath,
oldRegionInfo, false),
metaHLog,
conf,
oldDesc,
null);
region.initialize();
//Run major compaction to archive old stores
//to keep any snapshots to _acl_ unbroken
region.compactStores(true);
region.waitForFlushesAndCompactions();
region.close();
//Create new region dir
Path newRegionDir = new Path(newTablePath, newRegionInfo.getEncodedName());
if(!fs.exists(newRegionDir)) {
if(!fs.mkdirs(newRegionDir)) {
throw new IllegalStateException("Failed to create new region dir: " + newRegionDir);
}
}
//create new region info file, delete in case one exists
HRegionFileSystem.openRegionFromFileSystem(conf, fs, newTablePath, newRegionInfo, false);
//migrate region contents
for(FileStatus file : fs.listStatus(regionDir, new FSUtils.UserTableDirFilter(fs))) {
if(file.getPath().getName().equals(HRegionFileSystem.REGION_INFO_FILE))
continue;
if(!fs.rename(file.getPath(), newRegionDir)) {
throw new IllegalStateException("Failed to move file "+file.getPath()+" to " +
newRegionDir);
}
}
meta.put(MetaEditor.makePutFromRegionInfo(newRegionInfo));
meta.delete(MetaEditor.makeDeleteFromRegionInfo(oldRegionInfo));
}
} finally {
meta.flushcache();
meta.waitForFlushesAndCompactions();
meta.close();
metaHLog.closeAndDelete();
if(region != null) {
region.close();
}
}
if(!fs.rename(oldTablePath, backupDir)) {
throw new IllegalStateException("Failed to old data: "+oldTablePath+" to "+backupDir);
}
}
//Culled from FSTableDescriptors
private static HTableDescriptor readTableDescriptor(FileSystem fs,
FileStatus status) throws IOException {
int len = Ints.checkedCast(status.getLen());
byte [] content = new byte[len];
FSDataInputStream fsDataInputStream = fs.open(status.getPath());
try {
fsDataInputStream.readFully(content);
} finally {
fsDataInputStream.close();
}
HTableDescriptor htd = null;
try {
htd = HTableDescriptor.parseFrom(content);
} catch (DeserializationException e) {
throw new IOException("content=" + Bytes.toShort(content), e);
}
return htd;
}
private static final PathFilter TABLEINFO_PATHFILTER = new PathFilter() {
@Override
public boolean accept(Path p) {
// Accept any file that starts with TABLEINFO_NAME
return p.getName().startsWith(".tableinfo");
}
};
static final Comparator<FileStatus> TABLEINFO_FILESTATUS_COMPARATOR =
new Comparator<FileStatus>() {
@Override
public int compare(FileStatus left, FileStatus right) {
return -left.compareTo(right);
}};
// logic culled from FSTableDescriptors
static FileStatus getCurrentTableInfoStatus(FileSystem fs, Path dir)
throws IOException {
FileStatus [] status = FSUtils.listStatus(fs, dir, TABLEINFO_PATHFILTER);
if (status == null || status.length < 1) return null;
FileStatus mostCurrent = null;
for (FileStatus file : status) {
if (mostCurrent == null || TABLEINFO_FILESTATUS_COMPARATOR.compare(file, mostCurrent) < 0) {
mostCurrent = file;
}
}
return mostCurrent;
}
public static boolean verifyNSUpgrade(FileSystem fs, Path rootDir)

View File

@ -23,6 +23,7 @@ import java.io.DataInput;
import java.io.DataInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -38,6 +39,7 @@ 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.NamespaceDescriptor;
import org.apache.hadoop.hbase.catalog.MetaReader;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get;
@ -89,17 +91,17 @@ import com.google.protobuf.InvalidProtocolBufferException;
*/
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);
public static final TableName ACL_TABLE =
TableName.valueOf(ACL_TABLE_NAME);
public static final byte[] ACL_GLOBAL_NAME = ACL_TABLE_NAME;
public static final TableName ACL_TABLE_NAME =
TableName.valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "acl");
public static final byte[] ACL_GLOBAL_NAME = ACL_TABLE_NAME.getName();
/** 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);
public static final char NAMESPACE_PREFIX = '@';
/** Table descriptor for ACL internal table */
public static final HTableDescriptor ACL_TABLEDESC = new HTableDescriptor(ACL_TABLE);
public static final HTableDescriptor ACL_TABLEDESC = new HTableDescriptor(ACL_TABLE_NAME);
static {
ACL_TABLEDESC.addFamily(
new HColumnDescriptor(ACL_LIST_FAMILY,
@ -125,7 +127,7 @@ public class AccessControlLists {
* @param master reference to HMaster
*/
static void init(MasterServices master) throws IOException {
if (!MetaReader.tableExists(master.getCatalogTracker(), ACL_TABLE)) {
if (!MetaReader.tableExists(master.getCatalogTracker(), ACL_TABLE_NAME)) {
master.createTable(ACL_TABLEDESC, null);
}
}
@ -139,8 +141,8 @@ public class AccessControlLists {
static void addUserPermission(Configuration conf, UserPermission userPerm)
throws IOException {
Permission.Action[] actions = userPerm.getActions();
Put p = new Put(userPerm.isGlobal() ? ACL_GLOBAL_NAME : userPerm.getTable().getName());
byte[] rowKey = userPermissionRowKey(userPerm);
Put p = new Put(rowKey);
byte[] key = userPermissionKey(userPerm);
if ((actions == null) || (actions.length == 0)) {
@ -155,8 +157,8 @@ public class AccessControlLists {
}
p.add(ACL_LIST_FAMILY, key, value);
if (LOG.isDebugEnabled()) {
LOG.debug("Writing permission for table "+
userPerm.getTable()+" "+
LOG.debug("Writing permission with rowKey "+
Bytes.toString(rowKey)+" "+
Bytes.toString(key)+": "+Bytes.toStringBinary(value)
);
}
@ -184,8 +186,7 @@ public class AccessControlLists {
*/
static void removeUserPermission(Configuration conf, UserPermission userPerm)
throws IOException {
Delete d = new Delete(userPerm.isGlobal() ? ACL_GLOBAL_NAME : userPerm.getTable().getName());
Delete d = new Delete(userPermissionRowKey(userPerm));
byte[] key = userPermissionKey(userPerm);
if (LOG.isDebugEnabled()) {
@ -222,7 +223,27 @@ public class AccessControlLists {
}
/**
* Remove specified table column from the _acl_ table.
* Remove specified namespace from the acl table.
*/
static void removeNamespacePermissions(Configuration conf, String namespace)
throws IOException{
Delete d = new Delete(Bytes.toBytes(toNamespaceEntry(namespace)));
if (LOG.isDebugEnabled()) {
LOG.debug("Removing permissions of removed namespace "+ namespace);
}
HTable acls = null;
try {
acls = new HTable(conf, ACL_TABLE_NAME);
acls.delete(d);
} finally {
if (acls != null) acls.close();
}
}
/**
* Remove specified table column from the acl table.
*/
static void removeTablePermissions(Configuration conf, TableName tableName, byte[] column)
throws IOException{
@ -269,6 +290,18 @@ public class AccessControlLists {
}
}
static byte[] userPermissionRowKey(UserPermission userPerm) {
byte[] row;
if(userPerm.hasNamespace()) {
row = Bytes.toBytes(toNamespaceEntry(userPerm.getNamespace()));
} else if(userPerm.isGlobal()) {
row = ACL_GLOBAL_NAME;
} else {
row = userPerm.getTable().getName();
}
return row;
}
/**
* Build qualifier key from user permission:
* username
@ -295,14 +328,14 @@ public class AccessControlLists {
* metadata table.
*/
static boolean isAclRegion(HRegion region) {
return ACL_TABLE.equals(region.getTableDesc().getTableName());
return ACL_TABLE_NAME.equals(region.getTableDesc().getTableName());
}
/**
* Returns {@code true} if the given table is {@code _acl_} metadata table.
*/
static boolean isAclTable(HTableDescriptor desc) {
return ACL_TABLE.equals(desc.getTableName());
return ACL_TABLE_NAME.equals(desc.getTableName());
}
/**
@ -313,16 +346,16 @@ public class AccessControlLists {
* @return a map of the permissions for this table.
* @throws IOException
*/
static Map<TableName,ListMultimap<String,TablePermission>> loadAll(
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);
throw new IOException("Can only load permissions from "+ACL_TABLE_NAME);
}
Map<TableName,ListMultimap<String,TablePermission>> allPerms =
new TreeMap<TableName,ListMultimap<String,TablePermission>>();
Map<byte[], ListMultimap<String, TablePermission>> allPerms =
new TreeMap<byte[], ListMultimap<String, TablePermission>>();
// do a full scan of _acl_ table
@ -338,21 +371,21 @@ public class AccessControlLists {
boolean hasNext = iScanner.next(row);
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
TableName table = null;
byte[] entry = null;
for (KeyValue kv : row) {
if (table == null) {
table = TableName.valueOf(kv.getRow());
if (entry == null) {
entry = kv.getRow();
}
Pair<String,TablePermission> permissionsOfUserOnTable =
parseTablePermissionRecord(table, kv);
parsePermissionRecord(entry, kv);
if (permissionsOfUserOnTable != null) {
String username = permissionsOfUserOnTable.getFirst();
TablePermission permissions = permissionsOfUserOnTable.getSecond();
perms.put(username, permissions);
}
}
if (table != null) {
allPerms.put(table, perms);
if (entry != null) {
allPerms.put(entry, perms);
}
if (!hasNext) {
break;
@ -371,10 +404,10 @@ public class AccessControlLists {
* Load all permissions from the region server holding {@code _acl_},
* primarily intended for testing purposes.
*/
static Map<TableName,ListMultimap<String,TablePermission>> loadAll(
static Map<byte[], ListMultimap<String,TablePermission>> loadAll(
Configuration conf) throws IOException {
Map<TableName,ListMultimap<String,TablePermission>> allPerms =
new TreeMap<TableName,ListMultimap<String,TablePermission>>();
Map<byte[], ListMultimap<String,TablePermission>> allPerms =
new TreeMap<byte[], ListMultimap<String,TablePermission>>(Bytes.BYTES_RAWCOMPARATOR);
// do a full scan of _acl_, filtering on only first table region rows
@ -387,10 +420,9 @@ public class AccessControlLists {
acls = new HTable(conf, ACL_TABLE_NAME);
scanner = acls.getScanner(scan);
for (Result row : scanner) {
TableName tableName = TableName.valueOf(row.getRow());
ListMultimap<String,TablePermission> resultPerms =
parseTablePermissions(tableName, row);
allPerms.put(tableName, resultPerms);
parsePermissions(row.getRow(), row);
allPerms.put(row.getRow(), resultPerms);
}
} finally {
if (scanner != null) scanner.close();
@ -400,6 +432,16 @@ public class AccessControlLists {
return allPerms;
}
static ListMultimap<String, TablePermission> getTablePermissions(Configuration conf,
TableName tableName) throws IOException {
return getPermissions(conf, tableName != null ? tableName.getName() : null);
}
static ListMultimap<String, TablePermission> getNamespacePermissions(Configuration conf,
String namespace) throws IOException {
return getPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)));
}
/**
* Reads user permission assignments stored in the <code>l:</code> column
* family of the first table row in <code>_acl_</code>.
@ -409,23 +451,23 @@ public class AccessControlLists {
* used for storage.
* </p>
*/
static ListMultimap<String, TablePermission> getTablePermissions(Configuration conf,
TableName tableName) throws IOException {
if (tableName == null) tableName = ACL_TABLE;
static ListMultimap<String, TablePermission> getPermissions(Configuration conf,
byte[] entryName) throws IOException {
if (entryName == null) entryName = ACL_TABLE_NAME.getName();
// 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);
Get get = new Get(tableName.getName());
acls = new HTable(conf, ACL_TABLE_NAME);
Get get = new Get(entryName);
get.addFamily(ACL_LIST_FAMILY);
Result row = acls.get(get);
if (!row.isEmpty()) {
perms = parseTablePermissions(tableName, row);
perms = parsePermissions(entryName, row);
} else {
LOG.info("No permissions found in " + ACL_TABLE_NAME_STR + " for table "
+ tableName);
LOG.info("No permissions found in " + ACL_TABLE_NAME + " for acl entry "
+ Bytes.toString(entryName));
}
} finally {
if (acls != null) acls.close();
@ -438,11 +480,21 @@ public class AccessControlLists {
* Returns the currently granted permissions for a given table as a list of
* user plus associated permissions.
*/
static List<UserPermission> getUserTablePermissions(
Configuration conf, TableName tableName) throws IOException {
return getUserPermissions(conf, tableName.getName());
}
static List<UserPermission> getUserNamespacePermissions(
Configuration conf, String namespace) throws IOException {
return getUserPermissions(conf, Bytes.toBytes(toNamespaceEntry(namespace)));
}
static List<UserPermission> getUserPermissions(
Configuration conf, TableName tableName)
Configuration conf, byte[] entryName)
throws IOException {
ListMultimap<String,TablePermission> allPerms = getTablePermissions(
conf, tableName);
ListMultimap<String,TablePermission> allPerms = getPermissions(
conf, entryName);
List<UserPermission> perms = new ArrayList<UserPermission>();
@ -455,14 +507,14 @@ public class AccessControlLists {
return perms;
}
private static ListMultimap<String,TablePermission> parseTablePermissions(
TableName table, Result result) {
ListMultimap<String,TablePermission> perms = ArrayListMultimap.create();
private static ListMultimap<String, TablePermission> parsePermissions(
byte[] entryName, 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);
parsePermissionRecord(entryName, kv);
if (permissionsOfUserOnTable != null) {
String username = permissionsOfUserOnTable.getFirst();
@ -474,8 +526,8 @@ public class AccessControlLists {
return perms;
}
private static Pair<String,TablePermission> parseTablePermissionRecord(
TableName table, KeyValue kv) {
private static Pair<String, TablePermission> parsePermissionRecord(
byte[] entryName, KeyValue kv) {
// return X given a set of permissions encoded in the permissionRecord kv.
byte[] family = kv.getFamily();
@ -494,6 +546,15 @@ public class AccessControlLists {
// check for a column family appended to the key
// TODO: avoid the string conversion to make this more efficient
String username = Bytes.toString(key);
//Handle namespace entry
if(isNamespaceEntry(entryName)) {
return new Pair<String, TablePermission>(username,
new TablePermission(Bytes.toString(fromNamespaceEntry(entryName)), value));
}
//Handle table and global entry
//TODO global entry should be handled differently
int idx = username.indexOf(ACL_KEY_DELIMITER);
byte[] permFamily = null;
byte[] permQualifier = null;
@ -509,8 +570,8 @@ public class AccessControlLists {
}
}
return new Pair<String,TablePermission>(
username, new TablePermission(table, permFamily, permQualifier, value));
return new Pair<String,TablePermission>(username,
new TablePermission(TableName.valueOf(entryName), permFamily, permQualifier, value));
}
/**
@ -534,8 +595,8 @@ public class AccessControlLists {
if (ProtobufUtil.isPBMagicPrefix(data)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
try {
AccessControlProtos.UserTablePermissions perms =
AccessControlProtos.UserTablePermissions.newBuilder().mergeFrom(
AccessControlProtos.UsersAndPermissions perms =
AccessControlProtos.UsersAndPermissions.newBuilder().mergeFrom(
data, pblen, data.length - pblen).build();
return ProtobufUtil.toUserTablePermissions(perms);
} catch (InvalidProtocolBufferException e) {
@ -579,4 +640,37 @@ public class AccessControlLists {
return aclKey.substring(GROUP_PREFIX.length());
}
public static boolean isNamespaceEntry(String entryName) {
return entryName.charAt(0) == NAMESPACE_PREFIX;
}
public static boolean isNamespaceEntry(byte[] entryName) {
return entryName[0] == NAMESPACE_PREFIX;
}
public static String toNamespaceEntry(String namespace) {
return NAMESPACE_PREFIX + namespace;
}
public static String fromNamespaceEntry(String namespace) {
if(namespace.charAt(0) != NAMESPACE_PREFIX)
throw new IllegalArgumentException("Argument is not a valid namespace entry");
return namespace.substring(1);
}
public static byte[] toNamespaceEntry(byte[] namespace) {
byte[] ret = new byte[namespace.length+1];
ret[0] = NAMESPACE_PREFIX;
System.arraycopy(namespace, 0, ret, 1, namespace.length);
return ret;
}
public static byte[] fromNamespaceEntry(byte[] namespace) {
if(namespace[0] != NAMESPACE_PREFIX) {
throw new IllegalArgumentException("Argument is not a valid namespace entry: " +
Bytes.toString(namespace));
}
return Arrays.copyOfRange(namespace, 1, namespace.length);
}
}

View File

@ -26,6 +26,8 @@ import java.util.TreeSet;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
@ -139,16 +141,16 @@ public class AccessController extends BaseRegionObserver
void initialize(RegionCoprocessorEnvironment e) throws IOException {
final HRegion region = e.getRegion();
Map<TableName,ListMultimap<String,TablePermission>> tables =
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<TableName,ListMultimap<String,TablePermission>> t:
for (Map.Entry<byte[], ListMultimap<String,TablePermission>> t:
tables.entrySet()) {
TableName table = t.getKey();
byte[] entry = t.getKey();
ListMultimap<String,TablePermission> perms = t.getValue();
byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, e.getConfiguration());
this.authManager.getZKPermissionWatcher().writeToZookeeper(table, serialized);
this.authManager.getZKPermissionWatcher().writeToZookeeper(entry, serialized);
}
}
@ -159,8 +161,8 @@ public class AccessController extends BaseRegionObserver
*/
void updateACL(RegionCoprocessorEnvironment e,
final Map<byte[], List<? extends Cell>> familyMap) {
Set<TableName> tableSet =
new TreeSet<TableName>();
Set<byte[]> entries =
new TreeSet<byte[]>(Bytes.BYTES_RAWCOMPARATOR);
for (Map.Entry<byte[], List<? extends Cell>> f : familyMap.entrySet()) {
List<? extends Cell> cells = f.getValue();
for (Cell cell: cells) {
@ -168,21 +170,21 @@ public class AccessController extends BaseRegionObserver
if (Bytes.equals(kv.getBuffer(), kv.getFamilyOffset(),
kv.getFamilyLength(), AccessControlLists.ACL_LIST_FAMILY, 0,
AccessControlLists.ACL_LIST_FAMILY.length)) {
tableSet.add(TableName.valueOf(kv.getRow()));
entries.add(kv.getRow());
}
}
}
ZKPermissionWatcher zkw = this.authManager.getZKPermissionWatcher();
Configuration conf = regionEnv.getConfiguration();
for (TableName tableName: tableSet) {
for (byte[] entry: entries) {
try {
ListMultimap<String,TablePermission> perms =
AccessControlLists.getTablePermissions(conf, tableName);
AccessControlLists.getPermissions(conf, entry);
byte[] serialized = AccessControlLists.writePermissionsAsBytes(perms, conf);
zkw.writeToZookeeper(tableName, serialized);
zkw.writeToZookeeper(entry, serialized);
} catch (IOException ex) {
LOG.error("Failed updating permissions mirror for '" + tableName + "'", ex);
LOG.error("Failed updating permissions mirror for '" + Bytes.toString(entry) + "'",
ex);
}
}
}
@ -352,6 +354,35 @@ public class AccessController extends BaseRegionObserver
}
}
/**
* Authorizes that the current user has any of the given permissions for the
* given table, column family and column qualifier.
* @param namespace
* @throws IOException if obtaining the current user fails
* @throws AccessDeniedException if user has no authorization
*/
private void requirePermission(String request, String namespace,
Action... permissions) throws IOException {
User user = getActiveUser();
AuthResult result = null;
for (Action permission : permissions) {
if (authManager.authorize(user, namespace, permission)) {
result = AuthResult.allow(request, "Table permission granted", user,
permission, namespace);
break;
} else {
// rest of the world
result = AuthResult.deny(request, "Insufficient permissions", user,
permission, namespace);
}
}
logResult(result);
if (!result.isAllowed()) {
throw new AccessDeniedException("Insufficient permissions " + result.toContextString());
}
}
/**
* Authorizes that the current user has global privileges for the given action.
* @param perm The action being requested
@ -393,7 +424,7 @@ public class AccessController extends BaseRegionObserver
* being authorized, based on the given parameters.
* @param perm Action being requested
* @param tableName Affected table name.
* @param familiMap Affected column families.
* @param familyMap Affected column families.
*/
private void requireGlobalPermission(String request, Permission.Action perm, TableName tableName,
Map<byte[], ? extends Collection<byte[]>> familyMap) throws IOException {
@ -408,6 +439,26 @@ public class AccessController extends BaseRegionObserver
}
}
/**
* Checks that the user has the given global permission. The generated
* audit log message will contain context information for the operation
* being authorized, based on the given parameters.
* @param perm Action being requested
* @param namespace
*/
private void requireGlobalPermission(String request, Permission.Action perm,
String namespace) throws IOException {
User user = getActiveUser();
if (authManager.authorize(user, perm)) {
logResult(AuthResult.allow(request, "Global check allowed", user, perm, namespace));
} else {
logResult(AuthResult.deny(request, "Global check failed", user, perm, namespace));
throw new AccessDeniedException("Insufficient permissions for user '" +
(user != null ? user.getShortName() : "null") +"' (global, 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.
@ -632,7 +683,7 @@ public class AccessController extends BaseRegionObserver
throws IOException {
if (Bytes.equals(tableName.getName(), AccessControlLists.ACL_GLOBAL_NAME)) {
throw new AccessDeniedException("Not allowed to disable "
+ AccessControlLists.ACL_TABLE_NAME_STR + " table.");
+ AccessControlLists.ACL_TABLE_NAME + " table.");
}
requirePermission("disableTable", tableName, null, null, Action.ADMIN, Action.CREATE);
}
@ -780,6 +831,7 @@ public class AccessController extends BaseRegionObserver
@Override
public void preCreateNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns) throws IOException {
requireGlobalPermission("createNamespace", Action.ADMIN, ns.getName());
}
@Override
@ -788,18 +840,23 @@ public class AccessController extends BaseRegionObserver
}
@Override
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace) throws IOException {
public void preDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx, String namespace)
throws IOException {
requireGlobalPermission("deleteNamespace", Action.ADMIN, namespace);
}
@Override
public void postDeleteNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
String namespace) throws IOException {
AccessControlLists.removeNamespacePermissions(ctx.getEnvironment().getConfiguration(),
namespace);
LOG.info(namespace + "entry deleted in "+AccessControlLists.ACL_TABLE_NAME+" table.");
}
@Override
public void preModifyNamespace(ObserverContext<MasterCoprocessorEnvironment> ctx,
NamespaceDescriptor ns) throws IOException {
requireGlobalPermission("modifyNamespace", Action.ADMIN, ns.getName());
}
@Override
@ -1139,7 +1196,7 @@ public class AccessController extends BaseRegionObserver
action, e, Collections.EMPTY_MAP);
if (!authResult.isAllowed()) {
for(UserPermission userPerm:
AccessControlLists.getUserPermissions(regionEnv.getConfiguration(), tableName)) {
AccessControlLists.getUserTablePermissions(regionEnv.getConfiguration(), tableName)) {
for(Permission.Action userAction: userPerm.getActions()) {
if(userAction.equals(action)) {
return AuthResult.allow(method, "Access allowed", requestUser,
@ -1188,7 +1245,7 @@ public class AccessController extends BaseRegionObserver
public void grant(RpcController controller,
AccessControlProtos.GrantRequest request,
RpcCallback<AccessControlProtos.GrantResponse> done) {
UserPermission perm = ProtobufUtil.toUserPermission(request.getPermission());
UserPermission perm = ProtobufUtil.toUserPermission(request.getUserPermission());
AccessControlProtos.GrantResponse response = null;
try {
// verify it's only running at .acl.
@ -1197,7 +1254,15 @@ public class AccessController extends BaseRegionObserver
LOG.debug("Received request to grant access permission " + perm.toString());
}
requirePermission("grant", perm.getTable(), perm.getFamily(), perm.getQualifier(), Action.ADMIN);
switch(request.getUserPermission().getPermission().getType()) {
case Global :
case Table :
requirePermission("grant", perm.getTable(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN);
break;
case Namespace :
requireGlobalPermission("grant", Action.ADMIN, perm.getNamespace());
}
AccessControlLists.addUserPermission(regionEnv.getConfiguration(), perm);
if (AUDITLOG.isTraceEnabled()) {
@ -1206,7 +1271,7 @@ public class AccessController extends BaseRegionObserver
}
} else {
throw new CoprocessorException(AccessController.class, "This method "
+ "can only execute at " + Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
+ "can only execute at " + AccessControlLists.ACL_TABLE_NAME + " table.");
}
response = AccessControlProtos.GrantResponse.getDefaultInstance();
} catch (IOException ioe) {
@ -1220,7 +1285,7 @@ public class AccessController extends BaseRegionObserver
public void revoke(RpcController controller,
AccessControlProtos.RevokeRequest request,
RpcCallback<AccessControlProtos.RevokeResponse> done) {
UserPermission perm = ProtobufUtil.toUserPermission(request.getPermission());
UserPermission perm = ProtobufUtil.toUserPermission(request.getUserPermission());
AccessControlProtos.RevokeResponse response = null;
try {
// only allowed to be called on _acl_ region
@ -1229,8 +1294,15 @@ public class AccessController extends BaseRegionObserver
LOG.debug("Received request to revoke access permission " + perm.toString());
}
switch(request.getUserPermission().getPermission().getType()) {
case Global :
case Table :
requirePermission("revoke", perm.getTable(), perm.getFamily(),
perm.getQualifier(), Action.ADMIN);
break;
case Namespace :
requireGlobalPermission("revoke", Action.ADMIN, perm.getNamespace());
}
AccessControlLists.removeUserPermission(regionEnv.getConfiguration(), perm);
if (AUDITLOG.isTraceEnabled()) {
@ -1239,7 +1311,7 @@ public class AccessController extends BaseRegionObserver
}
} else {
throw new CoprocessorException(AccessController.class, "This method "
+ "can only execute at " + Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
+ "can only execute at " + AccessControlLists.ACL_TABLE_NAME + " table.");
}
response = AccessControlProtos.RevokeResponse.getDefaultInstance();
} catch (IOException ioe) {
@ -1254,21 +1326,30 @@ public class AccessController extends BaseRegionObserver
AccessControlProtos.UserPermissionsRequest request,
RpcCallback<AccessControlProtos.UserPermissionsResponse> done) {
AccessControlProtos.UserPermissionsResponse response = null;
try {
// only allowed to be called on _acl_ region
if (aclRegion) {
List<UserPermission> perms = null;
if(request.getType() == AccessControlProtos.Permission.Type.Table) {
TableName table = null;
if (request.hasTableName()) {
table = ProtobufUtil.toTableName(request.getTableName());
}
try {
// only allowed to be called on _acl_ region
if (aclRegion) {
requirePermission("userPermissions", table, null, null, Action.ADMIN);
List<UserPermission> perms = AccessControlLists.getUserPermissions(
perms = AccessControlLists.getUserTablePermissions(
regionEnv.getConfiguration(), table);
} else if (request.getType() == AccessControlProtos.Permission.Type.Namespace) {
perms = AccessControlLists.getUserNamespacePermissions(
regionEnv.getConfiguration(), request.getNamespaceName().toStringUtf8());
} else {
perms = AccessControlLists.getUserPermissions(
regionEnv.getConfiguration(), null);
}
response = ResponseConverter.buildUserPermissionsResponse(perms);
} else {
throw new CoprocessorException(AccessController.class, "This method "
+ "can only execute at " + Bytes.toString(AccessControlLists.ACL_TABLE_NAME) + " table.");
+ "can only execute at " + AccessControlLists.ACL_TABLE_NAME + " table.");
}
} catch (IOException ioe) {
// pass exception back up
@ -1371,7 +1452,8 @@ public class AccessController extends BaseRegionObserver
private boolean isSpecialTable(HRegionInfo regionInfo) {
TableName tableName = regionInfo.getTableName();
return tableName.equals(AccessControlLists.ACL_TABLE)
return tableName.equals(AccessControlLists.ACL_TABLE_NAME)
|| tableName.equals(TableName.NAMESPACE_TABLE_NAME)
|| tableName.equals(TableName.META_TABLE_NAME);
}

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.util.Bytes;
@InterfaceStability.Evolving
public class AuthResult {
private final boolean allowed;
private final String namespace;
private final TableName table;
private final Permission.Action action;
private final String request;
@ -58,6 +59,7 @@ public class AuthResult {
this.qualifier = qualifier;
this.action = action;
this.families = null;
this.namespace = null;
}
public AuthResult(boolean allowed, String request, String reason, User user,
@ -72,6 +74,21 @@ public class AuthResult {
this.qualifier = null;
this.action = action;
this.families = families;
this.namespace = null;
}
public AuthResult(boolean allowed, String request, String reason, User user,
Permission.Action action, String namespace) {
this.allowed = allowed;
this.request = request;
this.reason = reason;
this.user = user;
this.namespace = namespace;
this.action = action;
this.table = null;
this.family = null;
this.qualifier = null;
this.families = null;
}
public boolean isAllowed() {
@ -153,11 +170,13 @@ public class AuthResult {
.append(user != null ? user.getName() : "UNKNOWN")
.append(", ");
sb.append("scope=")
.append(table == null ? "GLOBAL" : table)
.append(", ");
sb.append("family=")
.append(namespace != null ? namespace : table == null ? "GLOBAL" : table);
if(namespace == null) {
sb.append(", ")
.append("family=")
.append(toFamilyString())
.append(", ");
}
sb.append("action=")
.append(action != null ? action.toString() : "")
.append(")");
@ -168,6 +187,11 @@ public class AuthResult {
return "AuthResult" + toContextString();
}
public static AuthResult allow(String request, String reason, User user,
Permission.Action action, String namespace) {
return new AuthResult(true, request, reason, user, action, namespace);
}
public static AuthResult allow(String request, String reason, User user,
Permission.Action action, TableName table, byte[] family, byte[] qualifier) {
return new AuthResult(true, request, reason, user, action, table, family, qualifier);
@ -179,6 +203,11 @@ public class AuthResult {
return new AuthResult(true, request, reason, user, action, table, families);
}
public static AuthResult deny(String request, String reason, User user,
Permission.Action action, String namespace) {
return new AuthResult(false, request, reason, user, action, namespace);
}
public static AuthResult deny(String request, String reason, User user,
Permission.Action action, TableName table, byte[] family, byte[] qualifier) {
return new AuthResult(false, request, reason, user, action, table, family, qualifier);

View File

@ -96,6 +96,9 @@ public class TableAuthManager {
private ConcurrentSkipListMap<TableName, PermissionCache<TablePermission>> tableCache =
new ConcurrentSkipListMap<TableName, PermissionCache<TablePermission>>();
private ConcurrentSkipListMap<String, PermissionCache<TablePermission>> nsCache =
new ConcurrentSkipListMap<String, PermissionCache<TablePermission>>();
private Configuration conf;
private ZKPermissionWatcher zkperms;
@ -147,7 +150,7 @@ public class TableAuthManager {
return this.zkperms;
}
public void refreshCacheFromWritable(TableName table,
public void refreshTableCacheFromWritable(TableName table,
byte[] data) throws IOException {
if (data != null && data.length > 0) {
ListMultimap<String,TablePermission> perms;
@ -169,6 +172,22 @@ public class TableAuthManager {
}
}
public void refreshNamespaceCacheFromWritable(String namespace, byte[] data) throws IOException {
if (data != null && data.length > 0) {
ListMultimap<String,TablePermission> perms;
try {
perms = AccessControlLists.readPermissions(data, conf);
} catch (DeserializationException e) {
throw new IOException(e);
}
if (perms != null) {
updateNsCache(namespace, perms);
}
} else {
LOG.debug("Skipping permission cache refresh because writable data is empty");
}
}
/**
* Updates the internal global permissions cache
*
@ -216,6 +235,29 @@ public class TableAuthManager {
tableCache.put(table, newTablePerms);
}
/**
* 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 namespace
* @param tablePerms
*/
private void updateNsCache(String namespace,
ListMultimap<String, TablePermission> tablePerms) {
PermissionCache<TablePermission> newTablePerms = new PermissionCache<TablePermission>();
for (Map.Entry<String, TablePermission> entry : tablePerms.entries()) {
if (AccessControlLists.isGroupPrincipal(entry.getKey())) {
newTablePerms.putGroup(AccessControlLists.getGroupName(entry.getKey()), entry.getValue());
} else {
newTablePerms.putUser(entry.getKey(), entry.getValue());
}
}
nsCache.put(namespace, newTablePerms);
}
private PermissionCache<TablePermission> getTablePermissions(TableName table) {
if (!tableCache.containsKey(table)) {
tableCache.putIfAbsent(table, new PermissionCache<TablePermission>());
@ -223,6 +265,13 @@ public class TableAuthManager {
return tableCache.get(table);
}
private PermissionCache<TablePermission> getNamespacePermissions(String namespace) {
if (!nsCache.containsKey(namespace)) {
nsCache.putIfAbsent(namespace, new PermissionCache<TablePermission>());
}
return nsCache.get(namespace);
}
/**
* Authorizes a global permission
* @param perms
@ -329,6 +378,45 @@ public class TableAuthManager {
return false;
}
public boolean authorize(User user, String namespace, Permission.Action action) {
if (authorizeUser(user.getShortName(), action)) {
return true;
}
PermissionCache<TablePermission> tablePerms = nsCache.get(namespace);
if (tablePerms != null) {
List<TablePermission> userPerms = tablePerms.getUser(user.getShortName());
if (authorize(userPerms, namespace, action)) {
return true;
}
String[] groupNames = user.getGroupNames();
if (groupNames != null) {
for (String group : groupNames) {
List<TablePermission> groupPerms = tablePerms.getGroup(group);
if (authorize(groupPerms, namespace, action)) {
return true;
}
}
}
}
return false;
}
private boolean authorize(List<TablePermission> perms, String namespace,
Permission.Action action) {
if (perms != null) {
for (TablePermission p : perms) {
if (p.implies(namespace, action)) {
return true;
}
}
} else if (LOG.isDebugEnabled()) {
LOG.debug("No permissions for authorize() check, table=" + namespace);
}
return false;
}
/**
* Checks global authorization for a specific action for a user, based on the
* stored user permissions.
@ -358,7 +446,7 @@ public class TableAuthManager {
if (authorizeUser(username, action)) {
return true;
}
if (table == null) table = AccessControlLists.ACL_TABLE;
if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
return authorize(getTablePermissions(table).getUser(username), table, family,
qualifier, action);
}
@ -387,7 +475,7 @@ public class TableAuthManager {
if (authorizeGroup(groupName, action)) {
return true;
}
if (table == null) table = AccessControlLists.ACL_TABLE;
if (table == null) table = AccessControlLists.ACL_TABLE_NAME;
return authorize(getTablePermissions(table).getGroup(groupName), table, family, action);
}
@ -481,11 +569,11 @@ public class TableAuthManager {
return false;
}
public void remove(byte[] table) {
remove(TableName.valueOf(table));
public void removeNamespace(byte[] ns) {
nsCache.remove(ns);
}
public void remove(TableName table) {
public void removeTable(TableName table) {
tableCache.remove(table);
}
@ -496,11 +584,11 @@ public class TableAuthManager {
* @param table
* @param perms
*/
public void setUserPermissions(String username, TableName table,
public void setTableUserPermissions(String username, TableName table,
List<TablePermission> perms) {
PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
tablePerms.replaceUser(username, perms);
writeToZooKeeper(table, tablePerms);
writeTableToZooKeeper(table, tablePerms);
}
/**
@ -510,20 +598,58 @@ public class TableAuthManager {
* @param table
* @param perms
*/
public void setGroupPermissions(String group, TableName table,
public void setTableGroupPermissions(String group, TableName table,
List<TablePermission> perms) {
PermissionCache<TablePermission> tablePerms = getTablePermissions(table);
tablePerms.replaceGroup(group, perms);
writeToZooKeeper(table, tablePerms);
writeTableToZooKeeper(table, tablePerms);
}
public void writeToZooKeeper(TableName table,
/**
* Overwrites the existing permission set for a given user for a table, and
* triggers an update for zookeeper synchronization.
* @param username
* @param namespace
* @param perms
*/
public void setNamespaceUserPermissions(String username, String namespace,
List<TablePermission> perms) {
PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
tablePerms.replaceUser(username, perms);
writeNamespaceToZooKeeper(namespace, tablePerms);
}
/**
* Overwrites the existing permission set for a group and triggers an update
* for zookeeper synchronization.
* @param group
* @param namespace
* @param perms
*/
public void setNamespaceGroupPermissions(String group, String namespace,
List<TablePermission> perms) {
PermissionCache<TablePermission> tablePerms = getNamespacePermissions(namespace);
tablePerms.replaceGroup(group, perms);
writeNamespaceToZooKeeper(namespace, tablePerms);
}
public void writeTableToZooKeeper(TableName table,
PermissionCache<TablePermission> tablePerms) {
byte[] serialized = new byte[0];
if (tablePerms != null) {
serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
}
zkperms.writeToZookeeper(table, serialized);
zkperms.writeToZookeeper(table.getName(), serialized);
}
public void writeNamespaceToZooKeeper(String namespace,
PermissionCache<TablePermission> tablePerms) {
byte[] serialized = new byte[0];
if (tablePerms != null) {
serialized = AccessControlLists.writePermissionsAsBytes(tablePerms.getAllPermissions(), conf);
}
zkperms.writeToZookeeper(Bytes.toBytes(AccessControlLists.toNamespaceEntry(namespace)),
serialized);
}
static Map<ZooKeeperWatcher,TableAuthManager> managerMap =

View File

@ -85,7 +85,11 @@ public class ZKPermissionWatcher extends ZooKeeperListener {
public void nodeDeleted(String path) {
if (aclZNode.equals(ZKUtil.getParent(path))) {
String table = ZKUtil.getNodeName(path);
authManager.remove(Bytes.toBytes(table));
if(AccessControlLists.isNamespaceEntry(table)) {
authManager.removeNamespace(Bytes.toBytes(table));
} else {
authManager.removeTable(TableName.valueOf(table));
}
}
}
@ -93,14 +97,14 @@ public class ZKPermissionWatcher extends ZooKeeperListener {
public void nodeDataChanged(String path) {
if (aclZNode.equals(ZKUtil.getParent(path))) {
// update cache on an existing table node
TableName table = TableName.valueOf(ZKUtil.getNodeName(path));
String entry = ZKUtil.getNodeName(path);
try {
byte[] data = ZKUtil.getDataAndWatch(watcher, path);
authManager.refreshCacheFromWritable(table, data);
refreshAuthManager(entry, data);
} catch (KeeperException ke) {
LOG.error("Error reading data from zookeeper for node "+table, ke);
LOG.error("Error reading data from zookeeper for node " + entry, ke);
// only option is to abort
watcher.abort("Zookeeper error getting data for node " + table, ke);
watcher.abort("Zookeeper error getting data for node " + entry, ke);
} catch (IOException ioe) {
LOG.error("Error reading permissions writables", ioe);
}
@ -126,36 +130,45 @@ public class ZKPermissionWatcher extends ZooKeeperListener {
for (ZKUtil.NodeAndData n : nodes) {
if (n.isEmpty()) continue;
String path = n.getNode();
TableName table = TableName.valueOf(ZKUtil.getNodeName(path));
String entry = (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(table, nodeData);
refreshAuthManager(entry, n.getData());
} catch (IOException ioe) {
LOG.error("Failed parsing permissions for table '" + table +
LOG.error("Failed parsing permissions for table '" + entry +
"' from zk", ioe);
}
}
}
private void refreshAuthManager(String entry, byte[] nodeData) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Updating permissions cache from node "+entry+" with data: "+
Bytes.toStringBinary(nodeData));
}
if(AccessControlLists.isNamespaceEntry(entry)) {
authManager.refreshNamespaceCacheFromWritable(
AccessControlLists.fromNamespaceEntry(entry), nodeData);
} else {
authManager.refreshTableCacheFromWritable(TableName.valueOf(entry), nodeData);
}
}
/***
* Write a table's access controls to the permissions mirror in zookeeper
* @param tableName
* @param entry
* @param permsData
*/
public void writeToZookeeper(TableName tableName, byte[] permsData) {
public void writeToZookeeper(byte[] entry, byte[] permsData) {
String entryName = Bytes.toString(entry);
String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE);
zkNode = ZKUtil.joinZNode(zkNode, tableName.getNameAsString());
zkNode = ZKUtil.joinZNode(zkNode, entryName);
try {
ZKUtil.createWithParents(watcher, zkNode);
ZKUtil.updateExistingNodeData(watcher, zkNode, permsData, -1);
} catch (KeeperException e) {
LOG.error("Failed updating permissions for table '" +
tableName + "'", e);
LOG.error("Failed updating permissions for entry '" +
entryName + "'", e);
watcher.abort("Failed writing node "+zkNode+" to zookeeper", e);
}
}

View File

@ -3822,7 +3822,8 @@ public class Hbase {
}
}
public static class createTable<I extends Iface> extends org.apache.thrift.ProcessFunction<I, createTable_args> {
public static class
createTable<I extends Iface> extends org.apache.thrift.ProcessFunction<I, createTable_args> {
public createTable() {
super("createTable");
}

View File

@ -36,26 +36,6 @@ module Hbase
# TODO: need to validate user name
# Verify that the specified permission is valid
if (permissions == nil || permissions.length == 0)
raise(ArgumentError, "Invalid permission: no actions associated with user")
end
if (table_name != nil)
# Table should exist
raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
tablebytes=table_name.to_java_bytes
htd = @admin.getTableDescriptor(tablebytes)
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)
end
begin
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config,
org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
@ -67,10 +47,48 @@ module Hbase
perm = org.apache.hadoop.hbase.security.access.Permission.new(
permissions.to_java_bytes)
# Verify that the specified permission is valid
if (permissions == nil || permissions.length == 0)
raise(ArgumentError, "Invalid permission: no actions associated with user")
end
if (table_name != nil)
#check if the tablename passed is actually a namespace
if (isNamespace?(table_name))
# Namespace should exist first.
namespace_name = table_name[1...table_name.length]
raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name)
#We pass the namespace name along with "@" so that we can differentiate a namespace from a table.
tablebytes=table_name.to_java_bytes
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant(
protocol, user, tablebytes, fambytes,
protocol, user, tablebytes, perm.getActions())
else
# Table should exist
raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
htd = @admin.getTableDescriptor(tablebytes)
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)
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant(
protocol, user, tableName, fambytes,
qualbytes, perm.getActions())
end
else
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.grant(
protocol, user, perm.getActions())
end
ensure
meta_table.close()
end
@ -82,21 +100,6 @@ module Hbase
# TODO: need to validate user name
if (table_name != nil)
# Table should exist
raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
tablebytes=table_name.to_java_bytes
htd = @admin.getTableDescriptor(tablebytes)
if (family != nil)
raise(ArgumentError, "Can't find 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)
end
begin
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config,
org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
@ -106,9 +109,41 @@ module Hbase
protocol = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos::
AccessControlService.newBlockingStub(service)
if (table_name != nil)
#check if the tablename passed is actually a namespace
if (isNamespace?(table_name))
# Namespace should exist first.
namespace_name = table_name[1...table_name.length]
raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name)
#We pass the namespace name along with "@" so that we can differentiate a namespace from a table.
tablebytes=table_name.to_java_bytes
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke(
protocol, user, tablebytes, fambytes, qualbytes)
protocol, user, tablebytes)
else
# Table should exist
raise(ArgumentError, "Can't find a table: #{table_name}") unless exists?(table_name)
tableName = org.apache.hadoop.hbase.TableName.valueOf(table_name.to_java_bytes)
htd = @admin.getTableDescriptor(tableName)
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)
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke(
protocol, user, tableName, fambytes, qualbytes)
end
else
# invoke cp endpoint to perform access controlse
org.apache.hadoop.hbase.protobuf.ProtobufUtil.revoke(
protocol, user)
end
ensure
meta_table.close()
end
@ -118,12 +153,6 @@ module Hbase
def user_permission(table_name=nil)
security_available?
if (table_name != nil)
raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
tablebytes=table_name.to_java_bytes
end
begin
meta_table = org.apache.hadoop.hbase.client.HTable.new(@config,
org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
@ -133,9 +162,23 @@ module Hbase
protocol = org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos::
AccessControlService.newBlockingStub(service)
# invoke cp endpoint to perform access controlse
if (table_name != nil)
#check if namespace is passed.
if (isNamespace?(table_name))
# Namespace should exist first.
namespace_name = table_name[1...table_name.length]
raise(ArgumentError, "Can't find a namespace: #{namespace_name}") unless namespace_exists?(namespace_name)
# invoke cp endpoint to perform access controls
perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions(
protocol, tablebytes)
protocol, table_name.to_java_bytes)
else
raise(ArgumentError, "Can't find table: #{table_name}") unless exists?(table_name)
perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions(
protocol, org.apache.hadoop.hbase.TableName.valueOf(table_name))
end
else
perms = org.apache.hadoop.hbase.protobuf.ProtobufUtil.getUserPermissions(protocol)
end
ensure
meta_table.close()
end
@ -167,11 +210,24 @@ module Hbase
@admin.tableExists(table_name)
end
def isNamespace?(table_name)
table_name.start_with?('@')
end
# Does Namespace exist
def namespace_exists?(namespace_name)
namespaceDesc = @admin.getNamespaceDescriptor(namespace_name)
if(namespaceDesc == nil)
return false
else
return true
end
end
# Make sure that security tables are available
def security_available?()
raise(ArgumentError, "DISABLED: Security features are not available") \
unless exists?(org.apache.hadoop.hbase.security.access.AccessControlLists::ACL_TABLE_NAME)
end
end
end

View File

@ -49,7 +49,7 @@ public class TestSecureLoadIncrementalHFiles extends TestLoadIncrementalHFiles{
util.startMiniCluster();
// Wait for the ACL table to become available
util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
}
}

View File

@ -54,7 +54,7 @@ public class TestSecureLoadIncrementalHFilesSplitRecovery extends TestLoadIncrem
util.startMiniCluster();
// Wait for the ACL table to become available
util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
util.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
}
//Disabling this test as it does not work in secure mode

View File

@ -23,6 +23,7 @@ import static org.junit.Assert.*;
import java.io.File;
import java.io.IOException;
import java.util.List;
import junit.framework.Assert;
@ -41,8 +42,11 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.HTable;
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.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.security.access.AccessControlLists;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.util.ToolRunner;
@ -64,6 +68,9 @@ import org.junit.experimental.categories.Category;
* Contains snapshots with snapshot{num}Keys as the contents:
* snapshot1Keys, snapshot2Keys
*
* Image also contains _acl_ table with one region and two storefiles.
* This is needed to test the acl table migration.
*
*/
@Category(MediumTests.class)
public class TestNamespaceUpgrade {
@ -103,6 +110,7 @@ public class TestNamespaceUpgrade {
Configuration toolConf = TEST_UTIL.getConfiguration();
conf.set(HConstants.HBASE_DIR, TEST_UTIL.getDefaultRootDirPath().toString());
ToolRunner.run(toolConf, new NamespaceUpgrade(), new String[]{"--upgrade"});
doFsCommand(shell, new String [] {"-lsr", "/"});
assertTrue(FSUtils.getVersion(fs, hbaseRootDir).equals(HConstants.FILE_SYSTEM_VERSION));
TEST_UTIL.startMiniHBaseCluster(1, 1);
@ -115,6 +123,22 @@ public class TestNamespaceUpgrade {
Assert.assertEquals(currentKeys.length, count);
}
assertEquals(2, TEST_UTIL.getHBaseAdmin().listNamespaceDescriptors().length);
//verify ACL table is migrated
HTable secureTable = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
ResultScanner scanner = secureTable.getScanner(new Scan());
int count = 0;
for(Result r : scanner) {
count++;
}
assertEquals(3, count);
assertFalse(TEST_UTIL.getHBaseAdmin().tableExists("_acl_"));
//verify ACL table was compacted
List<HRegion> regions = TEST_UTIL.getMiniHBaseCluster().getRegions(secureTable.getName());
for(HRegion region : regions) {
assertEquals(1, region.getStores().size());
}
}
private static File untar(final File testdir) throws IOException {

View File

@ -18,11 +18,25 @@
package org.apache.hadoop.hbase.security.access;
import static org.junit.Assert.fail;
import java.io.IOException;
import java.lang.reflect.UndeclaredThrowableException;
import java.security.PrivilegedActionException;
import java.security.PrivilegedExceptionAction;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.RetriesExhaustedWithDetailsException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.CheckPermissionsRequest;
import org.apache.hadoop.hbase.security.AccessDeniedException;
import org.apache.hadoop.hbase.security.User;
import com.google.protobuf.ServiceException;
/**
* Utility methods for testing security
*/
@ -37,4 +51,105 @@ public class SecureTestUtil {
String currentUser = User.getCurrent().getName();
conf.set("hbase.superuser", "admin,"+currentUser);
}
public void verifyAllowed(User user, PrivilegedExceptionAction... actions) throws Exception {
for (PrivilegedExceptionAction action : actions) {
try {
user.runAs(action);
} catch (AccessDeniedException ade) {
fail("Expected action to pass for user '" + user.getShortName() + "' but was denied");
}
}
}
public void verifyAllowed(PrivilegedExceptionAction action, User... users) throws Exception {
for (User user : users) {
verifyAllowed(user, action);
}
}
public void verifyDenied(User user, PrivilegedExceptionAction... actions) throws Exception {
for (PrivilegedExceptionAction action : actions) {
try {
user.runAs(action);
fail("Expected AccessDeniedException for user '" + user.getShortName() + "'");
} catch (IOException e) {
boolean isAccessDeniedException = false;
if(e instanceof RetriesExhaustedWithDetailsException) {
// in case of batch operations, and put, the client assembles a
// RetriesExhaustedWithDetailsException instead of throwing an
// AccessDeniedException
for(Throwable ex : ((RetriesExhaustedWithDetailsException) e).getCauses()) {
if (ex instanceof AccessDeniedException) {
isAccessDeniedException = true;
break;
}
}
}
else {
// For doBulkLoad calls AccessDeniedException
// is buried in the stack trace
Throwable ex = e;
do {
if (ex instanceof AccessDeniedException) {
isAccessDeniedException = true;
break;
}
} while((ex = ex.getCause()) != null);
}
if (!isAccessDeniedException) {
fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
}
} catch (UndeclaredThrowableException ute) {
// TODO why we get a PrivilegedActionException, which is unexpected?
Throwable ex = ute.getUndeclaredThrowable();
if (ex instanceof PrivilegedActionException) {
ex = ((PrivilegedActionException) ex).getException();
}
if (ex instanceof ServiceException) {
ServiceException se = (ServiceException)ex;
if (se.getCause() != null && se.getCause() instanceof AccessDeniedException) {
// expected result
return;
}
}
fail("Not receiving AccessDeniedException for user '" + user.getShortName() + "'");
}
}
}
public void verifyDenied(PrivilegedExceptionAction action, User... users) throws Exception {
for (User user : users) {
verifyDenied(user, action);
}
}
public void checkTablePerms(Configuration conf, byte[] table, byte[] family, byte[] column,
Permission.Action... actions) throws IOException {
Permission[] perms = new Permission[actions.length];
for (int i = 0; i < actions.length; i++) {
perms[i] = new TablePermission(TableName.valueOf(table), family, column, actions[i]);
}
checkTablePerms(conf, table, perms);
}
public void checkTablePerms(Configuration conf, byte[] table, Permission... perms) throws IOException {
CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
for (Permission p : perms) {
request.addPermission(ProtobufUtil.toPermission(p));
}
HTable acl = new HTable(conf, table);
try {
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(acl.coprocessorService(new byte[0]));
try {
protocol.checkPermissions(null, request.build());
} catch (ServiceException se) {
ProtobufUtil.toIOException(se);
}
} finally {
acl.close();
}
}
}

View File

@ -77,7 +77,7 @@ public class TestAccessControlFilter {
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.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
ADMIN = User.createUserForTesting(conf, "admin", new String[]{"supergroup"});
READER = User.createUserForTesting(conf, "reader", new String[0]);

View File

@ -18,8 +18,8 @@
package org.apache.hadoop.hbase.security.access;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@ -103,7 +103,7 @@ import com.google.protobuf.ServiceException;
*/
@Category(LargeTests.class)
@SuppressWarnings("rawtypes")
public class TestAccessController {
public class TestAccessController extends SecureTestUtil {
private static final Log LOG = LogFactory.getLog(TestAccessController.class);
@Rule public TestTableName TEST_TABLE = new TestTableName();
private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@ -156,7 +156,7 @@ public class TestAccessController {
Coprocessor.PRIORITY_HIGHEST, 1, conf);
// Wait for the ACL table to become available
TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
TEST_UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
// create a set of test users
SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
@ -196,7 +196,6 @@ public class TestAccessController {
AccessControlService.newBlockingStub(service);
protocol.grant(null, RequestConverter.buildGrantRequest(USER_ADMIN.getShortName(),
null, null, null,
AccessControlProtos.Permission.Action.ADMIN,
AccessControlProtos.Permission.Action.CREATE,
AccessControlProtos.Permission.Action.READ,
@ -235,17 +234,6 @@ public class TestAccessController {
assertEquals(0, AccessControlLists.getTablePermissions(conf, TEST_TABLE.getTableName()).size());
}
public void verifyAllowed(User user, PrivilegedExceptionAction... actions) throws Exception {
for (PrivilegedExceptionAction action : actions) {
try {
user.runAs(action);
} catch (AccessDeniedException ade) {
fail("Expected action to pass for user '" + user.getShortName() + "' but was denied: " +
ade.toString());
}
}
}
public void verifyAllowed(PrivilegedExceptionAction action, User... users) throws Exception {
for (User user : users) {
verifyAllowed(user, action);
@ -415,7 +403,7 @@ public class TestAccessController {
PrivilegedExceptionAction disableAclTable = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preDisableTable(ObserverContext.createAndPrepare(CP_ENV, null),
AccessControlLists.ACL_TABLE);
AccessControlLists.ACL_TABLE_NAME);
return null;
}
};
@ -1164,7 +1152,7 @@ public class TestAccessController {
ProtobufUtil.grant(protocol, tblUser.getShortName(),
tableName, null, null, Permission.Action.READ);
ProtobufUtil.grant(protocol, gblUser.getShortName(),
null, null, null, Permission.Action.READ);
Permission.Action.READ);
} finally {
acl.close();
}
@ -1188,7 +1176,7 @@ public class TestAccessController {
ProtobufUtil.grant(protocol, tblUser.getShortName(),
tableName, null, null, Permission.Action.WRITE);
ProtobufUtil.grant(protocol, gblUser.getShortName(),
null, null, null, Permission.Action.WRITE);
Permission.Action.WRITE);
} finally {
acl.close();
}
@ -1212,7 +1200,7 @@ public class TestAccessController {
ProtobufUtil.grant(protocol, tblUser.getShortName(), tableName, null, null,
Permission.Action.READ, Permission.Action.WRITE);
ProtobufUtil.revoke(protocol, tblUser.getShortName(), tableName, null, null);
ProtobufUtil.revoke(protocol, gblUser.getShortName(), null, null, null);
ProtobufUtil.revoke(protocol, gblUser.getShortName());
} finally {
acl.close();
}
@ -1236,7 +1224,7 @@ public class TestAccessController {
ProtobufUtil.grant(protocol, tblUser.getShortName(),
tableName, family1, null, Permission.Action.READ);
ProtobufUtil.grant(protocol, gblUser.getShortName(),
null, null, null, Permission.Action.READ);
Permission.Action.READ);
} finally {
acl.close();
}
@ -1262,7 +1250,7 @@ public class TestAccessController {
ProtobufUtil.grant(protocol, tblUser.getShortName(),
tableName, family2, null, Permission.Action.WRITE);
ProtobufUtil.grant(protocol, gblUser.getShortName(),
null, null, null, Permission.Action.WRITE);
Permission.Action.WRITE);
} finally {
acl.close();
}
@ -1287,7 +1275,7 @@ public class TestAccessController {
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(service);
ProtobufUtil.revoke(protocol, tblUser.getShortName(), tableName, family2, null);
ProtobufUtil.revoke(protocol, gblUser.getShortName(), null, null, null);
ProtobufUtil.revoke(protocol, gblUser.getShortName());
} finally {
acl.close();
}
@ -1607,12 +1595,12 @@ public class TestAccessController {
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(service);
perms = ProtobufUtil.getUserPermissions(protocol, null);
perms = ProtobufUtil.getUserPermissions(protocol);
} finally {
acl.close();
}
UserPermission adminPerm = new UserPermission(Bytes.toBytes(USER_ADMIN.getShortName()),
AccessControlLists.ACL_TABLE, null, null, Bytes.toBytes("ACRW"));
AccessControlLists.ACL_TABLE_NAME, null, null, Bytes.toBytes("ACRW"));
assertTrue("Only user admin has permission on table _acl_ per setup",
perms.size() == 1 && hasFoundUserPermission(adminPerm, perms));
}
@ -1632,7 +1620,10 @@ public class TestAccessController {
CheckPermissionsRequest.Builder request = CheckPermissionsRequest.newBuilder();
for (Action a : actions) {
request.addPermission(AccessControlProtos.Permission.newBuilder()
.addAction(ProtobufUtil.toPermissionAction(a)).build());
.setType(AccessControlProtos.Permission.Type.Global)
.setGlobalPermission(
AccessControlProtos.GlobalPermission.newBuilder()
.addAction(ProtobufUtil.toPermissionAction(a)).build()));
}
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
try {
@ -1813,8 +1804,11 @@ public class TestAccessController {
// check for wrong table region
CheckPermissionsRequest checkRequest = CheckPermissionsRequest.newBuilder()
.addPermission(AccessControlProtos.Permission.newBuilder()
.setType(AccessControlProtos.Permission.Type.Table)
.setTablePermission(
AccessControlProtos.TablePermission.newBuilder()
.setTableName(ProtobufUtil.toProtoTableName(TEST_TABLE.getTableName()))
.addAction(AccessControlProtos.Permission.Action.CREATE)
.addAction(AccessControlProtos.Permission.Action.CREATE))
).build();
acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
try {
@ -1935,7 +1929,7 @@ public class TestAccessController {
// User name for the new RegionServer we plan to add.
String activeUserForNewRs = currentUser + ".hfs."
+ hbaseCluster.getLiveRegionServerThreads().size();
ProtobufUtil.grant(protocol, activeUserForNewRs, null, null, null,
ProtobufUtil.grant(protocol, activeUserForNewRs,
Permission.Action.ADMIN, Permission.Action.CREATE,
Permission.Action.READ, Permission.Action.WRITE);
} finally {

View File

@ -0,0 +1,200 @@
/*
* 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.assertEquals;
import static org.junit.Assert.assertTrue;
import java.security.PrivilegedExceptionAction;
import java.util.List;
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.HConstants;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MediumTests;
import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HTable;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
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.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AccessControlProtos.AccessControlService;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.Permission.Action;
import org.apache.hadoop.hbase.util.Bytes;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import com.google.common.collect.ListMultimap;
import com.google.protobuf.BlockingRpcChannel;
@Category(MediumTests.class)
@SuppressWarnings("rawtypes")
public class TestNamespaceCommands extends SecureTestUtil {
private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
private static String TestNamespace = "ns1";
private static Configuration conf;
private static MasterCoprocessorEnvironment CP_ENV;
private static AccessController ACCESS_CONTROLLER;
//user with all permissions
private static User SUPERUSER;
// user with rw permissions
private static User USER_RW;
// user with create table permissions alone
private static User USER_CREATE;
// user with permission on namespace for testing all operations.
private static User USER_NSP_WRITE;
@BeforeClass
public static void beforeClass() throws Exception {
conf = UTIL.getConfiguration();
SecureTestUtil.enableSecurity(conf);
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY, AccessController.class.getName());
UTIL.startMiniCluster();
SUPERUSER = User.createUserForTesting(conf, "admin", new String[] { "supergroup" });
USER_RW = User.createUserForTesting(conf, "rw_user", new String[0]);
USER_CREATE = User.createUserForTesting(conf, "create_user", new String[0]);
USER_NSP_WRITE = User.createUserForTesting(conf, "namespace_write", new String[0]);
UTIL.getHBaseAdmin().createNamespace(NamespaceDescriptor.create(TestNamespace).build());
// Wait for the ACL table to become available
UTIL.waitTableAvailable(AccessControlLists.ACL_TABLE_NAME.getName(), 8000);
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
MasterCoprocessorHost cpHost = UTIL.getMiniHBaseCluster().getMaster().getCoprocessorHost();
cpHost.load(AccessController.class, Coprocessor.PRIORITY_HIGHEST, conf);
ACCESS_CONTROLLER = (AccessController) cpHost.findCoprocessor(AccessController.class.getName());
try {
BlockingRpcChannel service =
acl.coprocessorService(HConstants.EMPTY_START_ROW);
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(service);
ProtobufUtil.grant(protocol, USER_NSP_WRITE.getShortName(),
TestNamespace, Action.WRITE);
} finally {
acl.close();
}
}
@AfterClass
public static void afterClass() throws Exception {
UTIL.getHBaseAdmin().deleteNamespace(TestNamespace);
UTIL.shutdownMiniCluster();
}
@Test
public void testAclTableEntries() throws Exception {
String userTestNamespace = "userTestNsp";
AccessControlService.BlockingInterface protocol = null;
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
try {
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
protocol = AccessControlService.newBlockingStub(service);
ProtobufUtil.grant(protocol, userTestNamespace, TestNamespace, Permission.Action.WRITE);
Result result = acl.get(new Get(Bytes.toBytes(userTestNamespace)));
assertTrue(result != null);
ListMultimap<String, TablePermission> perms =
AccessControlLists.getNamespacePermissions(conf, TestNamespace);
assertEquals(2, perms.size());
List<TablePermission> namespacePerms = perms.get(userTestNamespace);
assertTrue(perms.containsKey(userTestNamespace));
assertEquals(1, namespacePerms.size());
assertEquals(TestNamespace,
namespacePerms.get(0).getNamespace());
assertEquals(null, namespacePerms.get(0).getFamily());
assertEquals(null, namespacePerms.get(0).getQualifier());
assertEquals(1, namespacePerms.get(0).getActions().length);
assertEquals(Permission.Action.WRITE, namespacePerms.get(0).getActions()[0]);
// Now revoke and check.
ProtobufUtil.revoke(protocol, userTestNamespace, TestNamespace,
Permission.Action.WRITE);
perms = AccessControlLists.getNamespacePermissions(conf, TestNamespace);
assertEquals(1, perms.size());
} finally {
acl.close();
}
}
@Test
public void testModifyNamespace() throws Exception {
PrivilegedExceptionAction modifyNamespace = new PrivilegedExceptionAction() {
public Object run() throws Exception {
ACCESS_CONTROLLER.preModifyNamespace(ObserverContext.createAndPrepare(CP_ENV, null),
NamespaceDescriptor.create(TestNamespace).addConfiguration("abc", "156").build());
return null;
}
};
// verify that superuser or hbase admin can modify namespaces.
verifyAllowed(modifyNamespace, SUPERUSER);
// all others should be denied
verifyDenied(modifyNamespace, USER_NSP_WRITE, USER_CREATE, USER_RW);
}
@Test
public void testGrantRevoke() throws Exception{
//Only HBase super user should be able to grant and revoke permissions to
// namespaces.
final String testUser = "testUser";
PrivilegedExceptionAction grantAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
try {
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(service);
ProtobufUtil.grant(protocol, testUser, TestNamespace, Action.WRITE);
} finally {
acl.close();
}
return null;
}
};
PrivilegedExceptionAction revokeAction = new PrivilegedExceptionAction() {
public Object run() throws Exception {
HTable acl = new HTable(conf, AccessControlLists.ACL_TABLE_NAME);
try {
BlockingRpcChannel service = acl.coprocessorService(HConstants.EMPTY_START_ROW);
AccessControlService.BlockingInterface protocol =
AccessControlService.newBlockingStub(service);
ProtobufUtil.revoke(protocol, testUser, TestNamespace, Action.WRITE);
} finally {
acl.close();
}
return null;
}
};
verifyAllowed(grantAction, SUPERUSER);
verifyDenied(grantAction, USER_CREATE, USER_RW);
verifyAllowed(revokeAction, SUPERUSER);
verifyDenied(revokeAction, USER_CREATE, USER_RW);
}
}

View File

@ -97,7 +97,7 @@ public class TestTablePermissions {
UTIL.startMiniCluster();
// Wait for the ACL table to become available
UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME);
UTIL.waitTableEnabled(AccessControlLists.ACL_TABLE_NAME.getName());
ZKW = new ZooKeeperWatcher(UTIL.getConfiguration(),
"TestTablePermissions", ABORTABLE);
@ -116,7 +116,7 @@ public class TestTablePermissions {
Configuration conf = UTIL.getConfiguration();
AccessControlLists.removeTablePermissions(conf, TEST_TABLE);
AccessControlLists.removeTablePermissions(conf, TEST_TABLE2);
AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE);
AccessControlLists.removeTablePermissions(conf, AccessControlLists.ACL_TABLE_NAME);
}
/**
@ -240,12 +240,12 @@ public class TestTablePermissions {
TablePermission.Action.READ, TablePermission.Action.WRITE));
// check full load
Map<TableName,ListMultimap<String,TablePermission>> allPerms =
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");
userPerms = allPerms.get(TEST_TABLE.getName()).get("hubert");
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
@ -253,7 +253,7 @@ public class TestTablePermissions {
assertEquals(1, permission.getActions().length);
assertEquals(TablePermission.Action.READ, permission.getActions()[0]);
userPerms = allPerms.get(TEST_TABLE2).get("hubert");
userPerms = allPerms.get(TEST_TABLE2.getName()).get("hubert");
assertNotNull(userPerms);
assertEquals(1, userPerms.size());
permission = userPerms.get(0);
@ -310,7 +310,7 @@ public class TestTablePermissions {
ListMultimap<String,TablePermission> permissions = createPermissions();
byte[] permsData = AccessControlLists.writePermissionsAsBytes(permissions, conf);
ListMultimap<String,TablePermission> copy =
ListMultimap<String, TablePermission> copy =
AccessControlLists.readPermissions(permsData, conf);
checkMultimapEqual(permissions, copy);

View File

@ -108,7 +108,7 @@ public class TestZKPermissionsWatcher {
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);
AUTH_A.setTableUserPermissions("george", TEST_TABLE, acl);
Thread.sleep(100);
// check it
@ -132,7 +132,7 @@ public class TestZKPermissionsWatcher {
// 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);
AUTH_B.setTableUserPermissions("hubert", TEST_TABLE, acl);
Thread.sleep(100);
// check it