diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index 88b01a24aa3e..8e5818ef47df 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -750,6 +750,17 @@ default void postSnapshot(final ObserverContext ct
final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
throws IOException {}
+ /**
+ * Called after the snapshot operation has been completed.
+ * @param ctx the environment to interact with the framework and master
+ * @param snapshot the SnapshotDescriptor for the snapshot
+ * @param tableDescriptor the TableDescriptor of the table to snapshot
+ */
+ default void postCompletedSnapshotAction(final ObserverContext ctx,
+ final SnapshotDescription snapshot, final TableDescriptor tableDescriptor)
+ throws IOException {
+ }
+
/**
* Called before listSnapshots request has been processed.
* @param ctx the environment to interact with the framework and master
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
index 36e086a596ae..01219d3d14e6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/FileLink.java
@@ -26,9 +26,6 @@
import java.io.FileNotFoundException;
import java.util.List;
-import org.apache.yetus.audience.InterfaceAudience;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
import org.apache.hadoop.fs.CanSetDropBehind;
import org.apache.hadoop.fs.CanSetReadahead;
import org.apache.hadoop.fs.CanUnbuffer;
@@ -40,6 +37,10 @@
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* The FileLink is a sort of hardlink, that allows access to a file given a set of locations.
@@ -297,6 +298,7 @@ public void unbuffer() {
* @throws IOException on unexpected error, or file not found.
*/
private FSDataInputStream tryOpen() throws IOException {
+ AccessControlException accessControlException = null;
for (Path path: fileLink.getLocations()) {
if (path.equals(currentPath)) continue;
try {
@@ -314,12 +316,24 @@ private FSDataInputStream tryOpen() throws IOException {
return(in);
} catch (FileNotFoundException e) {
// Try another file location
+ } catch (AccessControlException e) {
+ // Try another file location
+ accessControlException = e;
} catch (RemoteException re) {
- IOException ioe = re.unwrapRemoteException(FileNotFoundException.class);
- if (!(ioe instanceof FileNotFoundException)) throw re;
+ IOException ioe =
+ re.unwrapRemoteException(FileNotFoundException.class, AccessControlException.class);
+ if (ioe instanceof AccessControlException) {
+ accessControlException = (AccessControlException) ioe;
+ } else if (!(ioe instanceof FileNotFoundException)) {
+ throw re;
+ }
}
}
- throw new FileNotFoundException("Unable to open link: " + fileLink);
+ if (accessControlException != null) {
+ throw accessControlException;
+ } else {
+ throw new FileNotFoundException("Unable to open link: " + fileLink);
+ }
}
@Override
@@ -405,14 +419,22 @@ public Path getAvailablePath(FileSystem fs) throws IOException {
* @throws IOException on unexpected error.
*/
public FileStatus getFileStatus(FileSystem fs) throws IOException {
+ AccessControlException accessControlException = null;
for (int i = 0; i < locations.length; ++i) {
try {
return fs.getFileStatus(locations[i]);
} catch (FileNotFoundException e) {
// Try another file location
+ } catch (AccessControlException e) {
+ // Try another file location
+ accessControlException = e;
}
}
- throw new FileNotFoundException("Unable to open link: " + this);
+ if (accessControlException != null) {
+ throw accessControlException;
+ } else {
+ throw new FileNotFoundException("Unable to open link: " + this);
+ }
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 720f4967384c..d99ad559d75c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -1049,6 +1049,16 @@ public void call(MasterObserver observer) throws IOException {
});
}
+ public void postCompletedSnapshotAction(final SnapshotDescription snapshot,
+ final TableDescriptor hTableDescriptor) throws IOException {
+ execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
+ @Override
+ public void call(MasterObserver observer) throws IOException {
+ observer.postCompletedSnapshotAction(this, snapshot, hTableDescriptor);
+ }
+ });
+ }
+
public void preListSnapshot(final SnapshotDescription snapshot) throws IOException {
execOperation(coprocEnvironments.isEmpty() ? null : new MasterObserverOperation() {
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
index 0d652646a19c..a5f091b235d6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java
@@ -59,7 +59,7 @@
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
-
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
/**
@@ -228,6 +228,10 @@ public void process() {
status.markComplete(msg);
LOG.info(msg);
metricsSnapshot.addSnapshot(status.getCompletionTimestamp() - status.getStartTime());
+ if (master.getMasterCoprocessorHost() != null) {
+ master.getMasterCoprocessorHost()
+ .postCompletedSnapshotAction(ProtobufUtil.createSnapshotDesc(snapshot), this.htd);
+ }
} catch (Exception e) { // FindBugs: REC_CATCH_EXCEPTION
status.abort("Failed to complete snapshot " + snapshot.getName() + " on table " +
snapshotTable + " because " + e.getMessage());
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclController.java
new file mode 100644
index 000000000000..eafc4be34c72
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclController.java
@@ -0,0 +1,634 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellUtil;
+import org.apache.hadoop.hbase.HBaseInterfaceAudience;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Get;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+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.client.SnapshotDescription;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.coprocessor.CoreCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.HasMasterServices;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessor;
+import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
+import org.apache.hadoop.hbase.coprocessor.MasterObserver;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.security.UserProvider;
+import org.apache.hadoop.hbase.security.access.HDFSAclHelper.PathHelper;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Set HDFS ACLs to hFiles to make HBase granted users have permission to scan snapshot
+ *
+ * To use this feature, please mask sure HDFS config:
+ *
+ * - dfs.permissions.enabled = true
+ * - fs.permissions.umask-mode = 027 (or smaller umask than 027)
+ *
+ *
+ *
+ * The implementation of this feature is as followings:
+ *
+ * - For public directories such as 'data' and 'archive', set other permission to '--x' to make
+ * everyone have the permission to access the directory.
+ * - For namespace or table directories such as 'data/ns/table', 'archive/ns/table' and
+ * '.hbase-snapshot/snapshotName', set user 'r-x' acl and default 'r-x' acl when following
+ * operations happen:
+ *
+ * - grant user with global, namespace or table permission;
+ * - revoke user from global, namespace or table;
+ * - snapshot table;
+ * - truncate table;
+ *
+ *
+ * - Note: Because snapshots are at table level, so this feature just considers users with global,
+ * namespace or table permissions, ignores users with table CF or cell permissions.
+ *
+ *
+ */
+@CoreCoprocessor
+@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
+public class HDFSAclController implements MasterCoprocessor, MasterObserver {
+ private static final Logger LOG = LoggerFactory.getLogger(HDFSAclController.class);
+
+ public static final String HDFS_ACL_ENABLE = "hbase.hdfs.acl.enable";
+ public static final String HDFS_ACL_THREAD_NUMBER = "hbase.hdfs.acl.thread.number";
+ // the tmp directory to restore snapshot, it can not be a sub directory of HBase root dir
+ public static final String SNAPSHOT_RESTORE_TMP_DIR = "hbase.snapshot.restore.tmp.dir";
+ public static final String SNAPSHOT_RESTORE_TMP_DIR_DEFAULT =
+ "/hbase/.tmpdir-to-restore-snapshot";
+ // If enable this feature, set public directories permission to 751
+ public static final FsPermission ACL_ENABLE_PUBLIC_HFILE_PERMISSION =
+ new FsPermission((short) 0751);
+ // If enable this feature, set restore directory permission to 703
+ public static final FsPermission ACL_ENABLE_RESTORE_HFILE_PERMISSION =
+ new FsPermission((short) 0703);
+
+ private HDFSAclHelper hdfsAclHelper = null;
+ private PathHelper pathHelper = null;
+ private FileSystem fs = null;
+ /** Provider for mapping principal names to Users */
+ private UserProvider userProvider;
+
+ @Override
+ public Optional getMasterObserver() {
+ return Optional.of(this);
+ }
+
+ @Override
+ public void preMasterInitialization(final ObserverContext c)
+ throws IOException {
+ if (isHdfsAclEnabled(c.getEnvironment().getConfiguration())) {
+ MasterServices masterServices = null;
+ MasterCoprocessorEnvironment mEnv = c.getEnvironment();
+ if (mEnv instanceof HasMasterServices) {
+ masterServices = ((HasMasterServices) mEnv).getMasterServices();
+ }
+ if (masterServices == null) {
+ throw new RuntimeException("master services can not be null");
+ }
+ hdfsAclHelper = new HDFSAclHelper(masterServices);
+ pathHelper = hdfsAclHelper.getPathHelper();
+ fs = pathHelper.getFileSystem();
+ // Set public directory permission to 751 to make all users have access permission.
+ // And we also need the access permission of the parent of HBase root directory, but
+ // it's not set here, because the owner of HBase root directory may don't own permission
+ // to change it's parent permission to 751.
+ // The {root/.tmp} and {root/.tmp/data} directories are created to make global user HDFS
+ // acls can be inherited.
+ Path[] paths = new Path[] { pathHelper.getRootDir(), pathHelper.getDataDir(),
+ pathHelper.getTmpDir(), pathHelper.getTmpDataDir(), pathHelper.getArchiveDir(),
+ pathHelper.getArchiveDataDir(), pathHelper.getSnapshotRootDir() };
+ for (Path path : paths) {
+ if (!fs.exists(path)) {
+ fs.mkdirs(path);
+ }
+ fs.setPermission(path, ACL_ENABLE_PUBLIC_HFILE_PERMISSION);
+ }
+ // create snapshot restore directory
+ Path restoreDir = new Path(
+ mEnv.getConfiguration().get(SNAPSHOT_RESTORE_TMP_DIR, SNAPSHOT_RESTORE_TMP_DIR_DEFAULT));
+ if (!fs.exists(restoreDir)) {
+ fs.mkdirs(restoreDir);
+ fs.setPermission(restoreDir, ACL_ENABLE_RESTORE_HFILE_PERMISSION);
+ }
+ }
+ }
+
+ @Override
+ public void postStartMaster(ObserverContext ctx)
+ throws IOException {
+ try (Admin admin = ctx.getEnvironment().getConnection().getAdmin()) {
+ if (admin.tableExists(PermissionStorage.ACL_TABLE_NAME)) {
+ // check if hbase:acl table has 'm' CF
+ TableDescriptor tableDescriptor = admin.getDescriptor(PermissionStorage.ACL_TABLE_NAME);
+ boolean containHdfsAclFamily = Arrays.stream(tableDescriptor.getColumnFamilies())
+ .anyMatch(family -> Bytes.equals(family.getName(), HDFSAclStorage.HDFS_ACL_FAMILY));
+ if (!containHdfsAclFamily) {
+ TableDescriptorBuilder builder =
+ TableDescriptorBuilder.newBuilder(tableDescriptor).setColumnFamily(
+ ColumnFamilyDescriptorBuilder.newBuilder(HDFSAclStorage.HDFS_ACL_FAMILY).build());
+ admin.modifyTable(builder.build());
+ }
+ }
+ }
+ this.userProvider = UserProvider.instantiate(ctx.getEnvironment().getConfiguration());
+ }
+
+ @Override
+ public void preStopMaster(final ObserverContext c) {
+ if (hdfsAclHelper != null) {
+ hdfsAclHelper.stop();
+ }
+ }
+
+ @Override
+ public void postCompletedCreateTableAction(ObserverContext c,
+ TableDescriptor desc, RegionInfo[] regions) throws IOException {
+ if (hdfsAclHelper != null && !desc.getTableName().isSystemTable()) {
+ TableName tableName = desc.getTableName();
+ Path[] paths = new Path[] { pathHelper.getTmpTableDir(tableName),
+ pathHelper.getTableDir(tableName), pathHelper.getArchiveTableDir(tableName) };
+ for (Path path : paths) {
+ if (!fs.exists(path)) {
+ fs.mkdirs(path);
+ }
+ }
+ String owner = desc.getOwnerString();
+ if (owner == null) {
+ owner = getActiveUser(c).getShortName();
+ }
+ hdfsAclHelper.addTableAcl(desc.getTableName(), owner);
+ try (Table aclTable =
+ c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
+ HDFSAclStorage.addUserTableHdfsAcl(aclTable, owner, desc.getTableName());
+ }
+ }
+ }
+
+ @Override
+ public void postCreateNamespace(ObserverContext c,
+ NamespaceDescriptor ns) throws IOException {
+ if (hdfsAclHelper != null) {
+ Path[] nsDirs = new Path[] { pathHelper.getTmpNsDir(ns.getName()),
+ pathHelper.getArchiveNsDir(ns.getName()) };
+ for (Path nsDir : nsDirs) {
+ if (!fs.exists(nsDir)) {
+ fs.mkdirs(nsDir);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void postCompletedSnapshotAction(final ObserverContext c,
+ final SnapshotDescription snapshot, final TableDescriptor hTableDescriptor)
+ throws IOException {
+ if (hdfsAclHelper != null) {
+ hdfsAclHelper.snapshotAcl(snapshot);
+ }
+ }
+
+ @Override
+ public void postCompletedTruncateTableAction(
+ final ObserverContext c, TableName tableName)
+ throws IOException {
+ if (hdfsAclHelper != null && !tableName.isSystemTable()) {
+ hdfsAclHelper.resetTableAcl(tableName);
+ }
+ }
+
+ @Override
+ public void postDeleteTable(ObserverContext ctx,
+ TableName tableName) throws IOException {
+ if (hdfsAclHelper != null && !tableName.isSystemTable()) {
+ // remove table user HDFS acl from ns data directory if the user has no global/the ns/other
+ // tables of the ns permissions
+ Set removeUsers = new HashSet<>();
+ try (Table aclTable =
+ ctx.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
+ List users = HDFSAclStorage.getTableUsers(aclTable, tableName);
+ HDFSAclStorage.deleteTableHdfsAcl(aclTable, tableName);
+ for (String user : users) {
+ List userEntries = HDFSAclStorage.getUserEntries(aclTable, user);
+ boolean remove = true;
+ for (byte[] entry : userEntries) {
+ if (PermissionStorage.isGlobalEntry(entry)) {
+ remove = false;
+ break;
+ } else if (PermissionStorage.isNamespaceEntry(entry) && Bytes
+ .equals(PermissionStorage.fromNamespaceEntry(entry), tableName.getNamespace())) {
+ remove = false;
+ break;
+ } else if (Bytes.equals(TableName.valueOf(entry).getNamespace(),
+ tableName.getNamespace())) {
+ remove = false;
+ break;
+ }
+ }
+ if (remove) {
+ removeUsers.add(user);
+ }
+ }
+ }
+ if (removeUsers.size() > 0) {
+ hdfsAclHelper.removeNamespaceAcl(tableName, removeUsers);
+ }
+ }
+ }
+
+ @Override
+ public void postDeleteNamespace(ObserverContext ctx,
+ String namespace) throws IOException {
+ if (hdfsAclHelper != null) {
+ try (Table aclTable =
+ ctx.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
+ HDFSAclStorage.deleteNamespaceHdfsAcl(aclTable, namespace);
+ }
+ Path tmpNsDir = pathHelper.getTmpNsDir(namespace);
+ if (fs.exists(tmpNsDir)) {
+ if (fs.listStatus(tmpNsDir).length == 0) {
+ fs.delete(tmpNsDir, true);
+ } else {
+ LOG.error("The tmp directory of namespace {} is not empty after delete namespace",
+ namespace);
+ }
+ }
+ }
+ }
+
+ @Override
+ public void postGrant(ObserverContext c,
+ UserPermission userPermission, boolean mergeExistingPermissions) throws IOException {
+ if (hdfsAclHelper != null) {
+ try (Table aclTable =
+ c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
+ Configuration conf = c.getEnvironment().getConfiguration();
+ String userName = userPermission.getUser();
+ switch (userPermission.getAccessScope()) {
+ case GLOBAL:
+ UserPermission perm = getUserGlobalPermission(conf, userName);
+ if (containReadPermission(perm)) {
+ List userEntries = HDFSAclStorage.getUserEntries(aclTable, userName);
+ Set skipNamespaces = new HashSet<>();
+ Set skipTables = new HashSet<>();
+ for (byte[] entry : userEntries) {
+ if (PermissionStorage.isNamespaceEntry(entry)) {
+ skipNamespaces.add(PermissionStorage.fromNamespaceEntry(Bytes.toString(entry)));
+ } else if (!PermissionStorage.isGlobalEntry(entry)) {
+ skipTables.add(TableName.valueOf(entry));
+ }
+ }
+ hdfsAclHelper.grantAcl(userPermission, skipNamespaces, skipTables);
+ HDFSAclStorage.addUserGlobalHdfsAcl(aclTable, userName);
+ } else {
+ revokeUserGlobalPermission(aclTable, userName, userPermission);
+ }
+ break;
+ case NAMESPACE:
+ String namespace =
+ ((NamespacePermission) userPermission.getPermission()).getNamespace();
+ UserPermission nsPerm = getUserNamespacePermission(conf, userName, namespace);
+ if (containReadPermission(nsPerm)) {
+ if (!HDFSAclStorage.hasUserGlobalHdfsAcl(aclTable, userName)) {
+ List userEntries = HDFSAclStorage.getUserEntries(aclTable, userName);
+ Set skipTables = new HashSet<>();
+ for (byte[] entry : userEntries) {
+ if (!PermissionStorage.isNamespaceEntry(entry)
+ && !PermissionStorage.isGlobalEntry(entry)) {
+ skipTables.add(TableName.valueOf(entry));
+ }
+ }
+ hdfsAclHelper.grantAcl(userPermission, new HashSet<>(0), skipTables);
+ }
+ HDFSAclStorage.addUserNamespaceHdfsAcl(aclTable, userName, namespace);
+ } else {
+ revokeUserNamespacePermission(aclTable, userName, namespace, userPermission);
+ }
+ break;
+ case TABLE:
+ TableName tableName = ((TablePermission) userPermission.getPermission()).getTableName();
+ UserPermission tPerm = getUserTablePermission(conf, userName, tableName);
+ TablePermission tablePermission = (TablePermission) tPerm.getPermission();
+ if (tablePermission.hasFamily() || tablePermission.hasQualifier()) {
+ break;
+ }
+ if (containReadPermission(tPerm)) {
+ if (!HDFSAclStorage.hasUserGlobalHdfsAcl(aclTable, userName) && !HDFSAclStorage
+ .hasUserNamespaceHdfsAcl(aclTable, userName, tableName.getNamespaceAsString())) {
+ hdfsAclHelper.grantAcl(userPermission, new HashSet<>(0), new HashSet<>(0));
+ }
+ HDFSAclStorage.addUserTableHdfsAcl(aclTable, userName, tableName);
+ } else {
+ revokeUserTablePermission(aclTable, userName, tableName, userPermission);
+ }
+ break;
+ case EMPTY:
+ default:
+ }
+ }
+ }
+ }
+
+ @Override
+ public void postRevoke(ObserverContext c,
+ UserPermission userPermission) throws IOException {
+ if (hdfsAclHelper != null) {
+ Action[] actions = userPermission.getPermission().getActions();
+ if (actions == null || actions.length == 0
+ || Arrays.stream(actions).anyMatch(action -> action == Action.READ)) {
+ try (Table aclTable =
+ c.getEnvironment().getConnection().getTable(PermissionStorage.ACL_TABLE_NAME)) {
+ String userName = userPermission.getUser();
+ switch (userPermission.getAccessScope()) {
+ case GLOBAL:
+ revokeUserGlobalPermission(aclTable, userName, userPermission);
+ break;
+ case NAMESPACE:
+ NamespacePermission nsPerm = (NamespacePermission) userPermission.getPermission();
+ revokeUserNamespacePermission(aclTable, userName, nsPerm.getNamespace(),
+ userPermission);
+ break;
+ case TABLE:
+ TablePermission tPerm = (TablePermission) userPermission.getPermission();
+ if (tPerm.hasFamily() || tPerm.hasQualifier()) {
+ break;
+ }
+ revokeUserTablePermission(aclTable, userName, tPerm.getTableName(), userPermission);
+ break;
+ default:
+ }
+ }
+ }
+ }
+ }
+
+ private void revokeUserGlobalPermission(Table aclTable, String userName,
+ UserPermission userPermission) throws IOException {
+ // remove user global acls but reserve ns and table acls
+ Set skipNamespaces = new HashSet<>();
+ Set skipTables = new HashSet<>();
+ List userEntries = HDFSAclStorage.getUserEntries(aclTable, userName);
+ for (byte[] entry : userEntries) {
+ if (PermissionStorage.isNamespaceEntry(entry)) {
+ skipNamespaces.add(Bytes.toString(PermissionStorage.fromNamespaceEntry(entry)));
+ } else if (!PermissionStorage.isGlobalEntry(entry)) {
+ skipTables.add(TableName.valueOf(entry));
+ }
+ }
+ Set filterTableNames = new HashSet<>();
+ for (TableName tableName : skipTables) {
+ if (!skipNamespaces.contains(tableName.getNamespaceAsString())) {
+ filterTableNames.add(tableName);
+ }
+ }
+ hdfsAclHelper.revokeAcl(userPermission, skipNamespaces, filterTableNames);
+ HDFSAclStorage.deleteUserGlobalHdfsAcl(aclTable, userName);
+ }
+
+ private void revokeUserNamespacePermission(Table aclTable, String userName, String namespace,
+ UserPermission userPermission) throws IOException {
+ // remove user ns acls but reserve table acls
+ if (!HDFSAclStorage.hasUserGlobalHdfsAcl(aclTable, userName)) {
+ Set skipTables = new HashSet<>();
+ List userEntries = HDFSAclStorage.getUserEntries(aclTable, userName);
+ for (byte[] entry : userEntries) {
+ if (!PermissionStorage.isNamespaceEntry(entry) && !PermissionStorage.isGlobalEntry(entry)) {
+ skipTables.add(TableName.valueOf(entry));
+ }
+ }
+ hdfsAclHelper.revokeAcl(userPermission, new HashSet<>(), skipTables);
+ }
+ HDFSAclStorage.deleteUserNamespaceHdfsAcl(aclTable, userName, namespace);
+ }
+
+ private void revokeUserTablePermission(Table aclTable, String userName, TableName tableName,
+ UserPermission userPermission) throws IOException {
+ if (!HDFSAclStorage.hasUserGlobalHdfsAcl(aclTable, userName) && !HDFSAclStorage
+ .hasUserNamespaceHdfsAcl(aclTable, userName, tableName.getNamespaceAsString())) {
+ // remove table acls
+ hdfsAclHelper.revokeAcl(userPermission, new HashSet<>(0), new HashSet<>(0));
+ }
+ HDFSAclStorage.deleteUserTableHdfsAcl(aclTable, userName, tableName);
+ }
+
+ private boolean containReadPermission(UserPermission userPermission) {
+ if (userPermission != null) {
+ return Arrays.stream(userPermission.getPermission().getActions())
+ .anyMatch(action -> action == Permission.Action.READ);
+ }
+ return false;
+ }
+
+ private UserPermission getUserGlobalPermission(Configuration conf, String userName)
+ throws IOException {
+ List permissions = PermissionStorage.getUserPermissions(conf,
+ PermissionStorage.ACL_GLOBAL_NAME, null, null, userName, true);
+ if (permissions != null && permissions.size() > 0) {
+ return permissions.get(0);
+ }
+ return null;
+ }
+
+ private UserPermission getUserNamespacePermission(Configuration conf, String userName,
+ String namespace) throws IOException {
+ List permissions =
+ PermissionStorage.getUserNamespacePermissions(conf, namespace, userName, true);
+ if (permissions != null && permissions.size() > 0) {
+ return permissions.get(0);
+ }
+ return null;
+ }
+
+ private UserPermission getUserTablePermission(Configuration conf, String userName,
+ TableName tableName) throws IOException {
+ List permissions =
+ PermissionStorage.getUserTablePermissions(conf, tableName, null, null, userName, true);
+ if (permissions != null && permissions.size() > 0) {
+ return permissions.get(0);
+ }
+ return null;
+ }
+
+ private boolean isHdfsAclEnabled(Configuration configuration) {
+ return configuration.getBoolean(HDFS_ACL_ENABLE, false);
+ }
+
+ protected static final class HDFSAclStorage {
+ public static final byte[] HDFS_ACL_FAMILY = Bytes.toBytes("m");
+ private static final byte[] HDFS_ACL_VALUE = Bytes.toBytes("R");
+
+ static void addUserGlobalHdfsAcl(Table aclTable, String user) throws IOException {
+ addUserEntry(aclTable, user, PermissionStorage.ACL_GLOBAL_NAME);
+ }
+
+ static void addUserNamespaceHdfsAcl(Table aclTable, String user, String namespace)
+ throws IOException {
+ addUserEntry(aclTable, user, Bytes.toBytes(PermissionStorage.toNamespaceEntry(namespace)));
+ }
+
+ static void addUserTableHdfsAcl(Table aclTable, String user, TableName tableName)
+ throws IOException {
+ addUserEntry(aclTable, user, tableName.getName());
+ }
+
+ private static void addUserEntry(Table t, String user, byte[] entry) throws IOException {
+ Put p = new Put(entry);
+ p.addColumn(HDFS_ACL_FAMILY, Bytes.toBytes(user), HDFS_ACL_VALUE);
+ t.put(p);
+ }
+
+ static void deleteUserGlobalHdfsAcl(Table aclTable, String user) throws IOException {
+ deleteUserEntry(aclTable, user, PermissionStorage.ACL_GLOBAL_NAME);
+ }
+
+ static void deleteUserNamespaceHdfsAcl(Table aclTable, String user, String namespace)
+ throws IOException {
+ deleteUserEntry(aclTable, user, Bytes.toBytes(PermissionStorage.toNamespaceEntry(namespace)));
+ }
+
+ static void deleteUserTableHdfsAcl(Table aclTable, String user, TableName tableName)
+ throws IOException {
+ deleteUserEntry(aclTable, user, tableName.getName());
+ }
+
+ private static void deleteUserEntry(Table aclTable, String user, byte[] entry)
+ throws IOException {
+ Delete delete = new Delete(entry);
+ delete.addColumns(HDFS_ACL_FAMILY, Bytes.toBytes(user));
+ aclTable.delete(delete);
+ }
+
+ static void deleteNamespaceHdfsAcl(Table aclTable, String namespace) throws IOException {
+ deleteEntry(aclTable, Bytes.toBytes(PermissionStorage.toNamespaceEntry(namespace)));
+ }
+
+ static void deleteTableHdfsAcl(Table aclTable, TableName tableName) throws IOException {
+ deleteEntry(aclTable, tableName.getName());
+ }
+
+ private static void deleteEntry(Table aclTable, byte[] entry) throws IOException {
+ Delete delete = new Delete(entry);
+ delete.addFamily(HDFS_ACL_FAMILY);
+ aclTable.delete(delete);
+ }
+
+ static List getTableUsers(Table aclTable, TableName tableName) throws IOException {
+ return getEntryUsers(aclTable, tableName.getName());
+ }
+
+ private static List getEntryUsers(Table aclTable, byte[] entry) throws IOException {
+ List users = new ArrayList<>();
+ Get get = new Get(entry);
+ get.addFamily(HDFS_ACL_FAMILY);
+ Result result = aclTable.get(get);
+ List cells = result.listCells();
+ if (cells != null) {
+ for (Cell cell : cells) {
+ if (cell != null) {
+ users.add(Bytes.toString(CellUtil.cloneQualifier(cell)));
+ }
+ }
+ }
+ return users;
+ }
+
+ static List getUserEntries(Table aclTable, String userName) throws IOException {
+ Scan scan = new Scan();
+ scan.addColumn(HDFS_ACL_FAMILY, Bytes.toBytes(userName));
+ ResultScanner scanner = aclTable.getScanner(scan);
+ List entry = new ArrayList<>();
+ for (Result result : scanner) {
+ if (result != null && result.getRow() != null) {
+ entry.add(result.getRow());
+ }
+ }
+ return entry;
+ }
+
+ static boolean hasUserGlobalHdfsAcl(Table aclTable, String user) throws IOException {
+ return hasUserEntry(aclTable, user, PermissionStorage.ACL_GLOBAL_NAME);
+ }
+
+ static boolean hasUserNamespaceHdfsAcl(Table aclTable, String user, String namespace)
+ throws IOException {
+ return hasUserEntry(aclTable, user,
+ Bytes.toBytes(PermissionStorage.toNamespaceEntry(namespace)));
+ }
+
+ static boolean hasUserTableHdfsAcl(Table aclTable, String user, TableName tableName)
+ throws IOException {
+ return hasUserEntry(aclTable, user, tableName.getName());
+ }
+
+ private static boolean hasUserEntry(Table aclTable, String userName, byte[] entry)
+ throws IOException {
+ Get get = new Get(entry);
+ get.addColumn(HDFS_ACL_FAMILY, Bytes.toBytes(userName));
+ Result result = aclTable.get(get);
+ if (result != null) {
+ byte[] value = result.getValue(HDFS_ACL_FAMILY, Bytes.toBytes(userName));
+ if (value != null && Bytes.equals(value, HDFS_ACL_VALUE)) {
+ return true;
+ }
+ }
+ return false;
+ }
+ }
+
+ private User getActiveUser(ObserverContext> ctx) throws IOException {
+ // for non-rpc handling, fallback to system user
+ Optional optionalUser = ctx.getCaller();
+ if (optionalUser.isPresent()) {
+ return optionalUser.get();
+ }
+ return userProvider.getCurrent();
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclHelper.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclHelper.java
new file mode 100644
index 000000000000..5e7d4ecce6d5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/HDFSAclHelper.java
@@ -0,0 +1,649 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
+import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
+import static org.apache.hadoop.fs.permission.AclEntryType.USER;
+import static org.apache.hadoop.fs.permission.FsAction.READ_EXECUTE;
+import static org.apache.hadoop.hbase.security.access.Permission.Action.READ;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryScope;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.SnapshotDescription;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
+import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos;
+
+/**
+ * A helper to set HBase granted user access acl and default acl over hFiles.
+ */
+@InterfaceAudience.Private
+public class HDFSAclHelper {
+ private static final Logger LOG = LoggerFactory.getLogger(HDFSAclHelper.class);
+ private final MasterServices masterServices;
+ private final Configuration conf;
+ private FileSystem fs;
+ private PathHelper pathHelper;
+ private ExecutorService pool;
+
+ public HDFSAclHelper(MasterServices masterServices) throws IOException {
+ this.masterServices = masterServices;
+ this.conf = masterServices.getConfiguration();
+ this.pathHelper = new PathHelper(conf);
+ this.fs = pathHelper.getFileSystem();
+ this.pathHelper = new PathHelper(conf);
+ this.pool =
+ Executors.newFixedThreadPool(conf.getInt(HDFSAclController.HDFS_ACL_THREAD_NUMBER, 10),
+ new ThreadFactoryBuilder().setNameFormat("hdfs-acl-thread-%d").setDaemon(true).build());
+ }
+
+ public void stop() {
+ if (pool != null) {
+ pool.shutdown();
+ }
+ }
+
+ /**
+ * Set acl when grant user permission
+ * @param userPerm the user and permission
+ * @param skipNamespaces the namespace set to skip set acl because already set
+ * @param skipTables the table set to skip set acl because already set
+ * @return false if an error occurred, otherwise true
+ */
+ public boolean grantAcl(UserPermission userPerm, Set skipNamespaces,
+ Set skipTables) {
+ try {
+ long start = System.currentTimeMillis();
+ setHDFSAcl(getHdfsAclOperations(userPerm, HDFSAclOperation.OperationType.MODIFY,
+ skipNamespaces, skipTables));
+ LOG.info("Set HDFS acl when grant {}, cost {} ms", userPerm.toString(),
+ System.currentTimeMillis() - start);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when grant: {}", userPerm != null ? userPerm.toString() : null,
+ e);
+ return false;
+ }
+ }
+
+ /**
+ * Remove acl when grant or revoke user permission
+ * @param userPerm the user and permission
+ * @param skipNamespaces the namespace set to skip remove acl
+ * @param skipTables the table set to skip remove acl
+ * @return false if an error occurred, otherwise true
+ */
+ public boolean revokeAcl(UserPermission userPerm, Set skipNamespaces,
+ Set skipTables) {
+ try {
+ long start = System.currentTimeMillis();
+ setHDFSAcl(getHdfsAclOperations(userPerm, HDFSAclOperation.OperationType.REMOVE,
+ skipNamespaces, skipTables));
+ LOG.info("Set HDFS acl when revoke {}, cost {} ms", userPerm.toString(),
+ System.currentTimeMillis() - start);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when revoke: {}", userPerm != null ? userPerm.toString() : null,
+ e);
+ return false;
+ }
+ }
+
+ /**
+ * Set acl when take a snapshot
+ * @param snapshot the snapshot desc
+ * @return false if an error occurred, otherwise true
+ */
+ public boolean snapshotAcl(SnapshotDescription snapshot) {
+ try {
+ long start = System.currentTimeMillis();
+ TableName tableName = snapshot.getTableName();
+ // global user permission can be inherited from default acl automatically
+ Set userSet = getUsersWithTableReadAction(tableName);
+ userSet.addAll(getUsersWithNamespaceReadAction(tableName.getNamespaceAsString()));
+ Path path = pathHelper.getSnapshotDir(snapshot.getName());
+ List operations = new ArrayList<>(1);
+ operations.add(new HDFSAclOperation(fs, path, userSet, HDFSAclOperation.OperationType.MODIFY,
+ READ_EXECUTE, true, new ArrayList<>()));
+ setHDFSAcl(operations);
+ LOG.info("Set HDFS acl when snapshot {}, cost {} ms", snapshot.getName(),
+ System.currentTimeMillis() - start);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when snapshot {}", snapshot != null ? snapshot.getName() : null,
+ e);
+ return false;
+ }
+ }
+
+ /**
+ * Reset acl when truncate table
+ * @param tableName the specific table
+ * @return false if an error occurred, otherwise true
+ */
+ public boolean resetTableAcl(TableName tableName) {
+ try {
+ long start = System.currentTimeMillis();
+ Set users = getUsersWithTableReadAction(tableName);
+ setHDFSAcl(
+ getTableHdfsAclOperations(users, HDFSAclOperation.OperationType.MODIFY, tableName));
+ LOG.info("Set HDFS acl when truncate {}, cost {} ms", tableName,
+ System.currentTimeMillis() - start);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when truncate {}", tableName != null ? tableName : null, e);
+ return false;
+ }
+ }
+
+ /**
+ * Remove table acl from ns dir when delete table
+ * @param tableName the table
+ * @param removeUsers the users whose acl will be removed
+ * @return false if an error occurred, otherwise true
+ */
+ public boolean removeNamespaceAcl(TableName tableName, Set removeUsers) {
+ try {
+ long start = System.currentTimeMillis();
+ List aclEntries = removeUsers.stream()
+ .map(removeUser -> new AclEntry.Builder().setScope(AclEntryScope.ACCESS).setType(USER)
+ .setName(removeUser).setPermission(FsAction.READ_EXECUTE).build())
+ .collect(Collectors.toList());
+ Path nsPath = pathHelper.getDataNsDir(tableName.getNamespaceAsString());
+ fs.removeAclEntries(nsPath, aclEntries);
+ LOG.info("Remove HDFS acl when delete table {}, cost {} ms", tableName,
+ System.currentTimeMillis() - start);
+ return true;
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when delete table {}", tableName != null ? tableName : null, e);
+ return false;
+ }
+ }
+
+ /**
+ * Set table owner acl when create table
+ * @param tableName the table
+ * @param user the table owner
+ */
+ public void addTableAcl(TableName tableName, String user) {
+ try {
+ long start = System.currentTimeMillis();
+ List aclEntries = new ArrayList<>(2);
+ AclEntry accessAclEntry = new AclEntry.Builder().setScope(AclEntryScope.ACCESS).setType(USER)
+ .setName(user).setPermission(FsAction.READ_EXECUTE).build();
+ aclEntries.add(accessAclEntry);
+ aclEntries.add(new AclEntry.Builder().setScope(AclEntryScope.DEFAULT).setType(USER)
+ .setName(user).setPermission(FsAction.READ_EXECUTE).build());
+ // set access and default HDFS acl for table dir
+ fs.modifyAclEntries(pathHelper.getTmpTableDir(tableName), aclEntries);
+ fs.modifyAclEntries(pathHelper.getArchiveTableDir(tableName), aclEntries);
+ Path tableDir = pathHelper.getTableDir(tableName);
+ HDFSAclOperation operation = new HDFSAclOperation(fs, tableDir, Sets.newHashSet(user),
+ HDFSAclOperation.OperationType.MODIFY, READ_EXECUTE, true, new ArrayList<>(0));
+ setSingleHDFSAcl(operation).get();
+ // set access HDFS acl for ns dir
+ List nsPath = getDefaultNamespacePath(tableName.getNamespaceAsString(), false);
+ for (Path path : nsPath) {
+ fs.modifyAclEntries(path, Lists.newArrayList(accessAclEntry));
+ }
+ LOG.info("Set HDFS acl when create table {}, cost {} ms", tableName,
+ System.currentTimeMillis() - start);
+ } catch (Exception e) {
+ LOG.error("Set HDFS acl error when create table {}", tableName != null ? tableName : null, e);
+ }
+ }
+
+ /**
+ * Generate the {@link HDFSAclOperation} list
+ * @param userPermission the user and permission
+ * @param operationType MODIFY or REMOVE HDFS acl
+ * @param skipNamespaces a namespace list whose related paths will be skipped when set HDFS acl
+ * @param skipTables a table list whose related paths will be skipped when set HDFS acl
+ * @return the {@link HDFSAclOperation} list
+ * @throws IOException if an error occurred
+ */
+ private List getHdfsAclOperations(UserPermission userPermission,
+ HDFSAclOperation.OperationType operationType, Set skipNamespaces,
+ Set skipTables) throws IOException {
+ List hdfsAclOperations = new ArrayList<>();
+ Set users = Sets.newHashSet(userPermission.getUser());
+ switch (userPermission.getAccessScope()) {
+ case GLOBAL:
+ hdfsAclOperations
+ .addAll(getGlobalHdfsAclOperations(users, operationType, skipNamespaces, skipTables));
+ break;
+ case NAMESPACE:
+ NamespacePermission namespacePermission =
+ (NamespacePermission) userPermission.getPermission();
+ hdfsAclOperations.addAll(getNamespaceHdfsAclOperations(users, operationType,
+ namespacePermission.getNamespace(), skipTables));
+ break;
+ case TABLE:
+ TablePermission tablePermission = (TablePermission) userPermission.getPermission();
+ hdfsAclOperations.addAll(
+ getTableHdfsAclOperations(users, operationType, tablePermission.getTableName()));
+ break;
+ default:
+ LOG.error("Unknown scope for permission {}", userPermission);
+ }
+ return hdfsAclOperations;
+ }
+
+ private List getGlobalHdfsAclOperations(Set users,
+ HDFSAclOperation.OperationType operationType, Set skipNamespaces,
+ Set skipTables) throws IOException {
+ // default acl path
+ List defaultGlobalPathList = getDefaultGlobalPath();
+ // skip namespace path
+ List skipPaths = Lists.newArrayList();
+ for (String namespace : skipNamespaces) {
+ skipPaths.addAll(getDefaultNamespacePath(namespace, true));
+ }
+ // skip table path
+ List tableNsPathList = new ArrayList<>();
+ for (TableName tableName : skipTables) {
+ skipPaths.addAll(getDefaultTablePath(tableName, true));
+ tableNsPathList.addAll(getDefaultNamespacePath(tableName.getNamespaceAsString(), false));
+ }
+ List hdfsAclOperations = defaultGlobalPathList.stream().map(
+ path -> new HDFSAclOperation(fs, path, users, operationType, READ_EXECUTE, true, skipPaths))
+ .collect(Collectors.toList());
+ if (operationType == HDFSAclOperation.OperationType.REMOVE) {
+ // add ns access acl for skip table users
+ hdfsAclOperations.addAll(tableNsPathList
+ .stream().map(path -> new HDFSAclOperation(fs, path, users,
+ HDFSAclOperation.OperationType.MODIFY, READ_EXECUTE, false, new ArrayList<>()))
+ .collect(Collectors.toList()));
+ }
+ return hdfsAclOperations;
+ }
+
+ private List getNamespaceHdfsAclOperations(Set users,
+ HDFSAclOperation.OperationType operationType, String namespace, Set skipTables)
+ throws IOException {
+ // default acl path
+ List defaultNsPathList = getDefaultNamespacePath(namespace, true);
+ // skip path
+ List skipTablePaths = Lists.newArrayList();
+ List skipNsPaths = new ArrayList<>();
+ for (TableName tableName : skipTables) {
+ skipTablePaths.addAll(getDefaultTablePath(tableName, true));
+ skipNsPaths.addAll(getDefaultNamespacePath(tableName.getNamespaceAsString(), false));
+ }
+ List hdfsAclOperations =
+ defaultNsPathList.stream().map(path -> new HDFSAclOperation(fs, path, users, operationType,
+ READ_EXECUTE, true, skipTablePaths)).collect(Collectors.toList());
+ if (operationType == HDFSAclOperation.OperationType.REMOVE) {
+ // add ns access acl for skip table users
+ hdfsAclOperations.addAll(skipNsPaths
+ .stream().map(path -> new HDFSAclOperation(fs, path, users,
+ HDFSAclOperation.OperationType.MODIFY, READ_EXECUTE, false, new ArrayList<>()))
+ .collect(Collectors.toList()));
+ }
+ return hdfsAclOperations;
+ }
+
+ private List getTableHdfsAclOperations(Set users,
+ HDFSAclOperation.OperationType operationType, TableName tableName) throws IOException {
+ String tableNamespace = tableName.getNamespaceAsString();
+ // acl path
+ List tableNsPathList = getDefaultNamespacePath(tableNamespace, false);
+ // default acl path
+ List tablePathList = getDefaultTablePath(tableName, true);
+ // generate hdfs acl operations
+ List hdfsAclOperations =
+ tableNsPathList.stream().map(path -> new HDFSAclOperation(fs, path, users, operationType,
+ READ_EXECUTE, false, new ArrayList<>())).collect(Collectors.toList());
+ hdfsAclOperations.addAll(tablePathList.stream().map(path -> new HDFSAclOperation(fs, path,
+ users, operationType, READ_EXECUTE, true, new ArrayList<>())).collect(Collectors.toList()));
+ return hdfsAclOperations;
+ }
+
+ /**
+ * return paths that user will global permission will visit
+ * @return the path list
+ */
+ private List getDefaultGlobalPath() {
+ return Lists.newArrayList(pathHelper.getTmpDataDir(), pathHelper.getDataDir(),
+ pathHelper.getArchiveDataDir(), pathHelper.getSnapshotRootDir());
+ }
+
+ /**
+ * return paths that user will namespace permission will visit
+ * @param namespace the namespace
+ * @param includeNamespaceSnapshot true if return the paths of namespace snapshots
+ * @return the path list
+ * @throws IOException if an error occurred
+ */
+ private List getDefaultNamespacePath(String namespace, boolean includeNamespaceSnapshot)
+ throws IOException {
+ List paths = Lists.newArrayList(pathHelper.getTmpNsDir(namespace),
+ pathHelper.getDataNsDir(namespace), pathHelper.getArchiveNsDir(namespace));
+ if (includeNamespaceSnapshot) {
+ paths.addAll(getNamespaceSnapshots(namespace).stream()
+ .map(snap -> pathHelper.getSnapshotDir(snap)).collect(Collectors.toList()));
+ }
+ return paths;
+ }
+
+ /**
+ * return paths that user will table permission will visit
+ * @param tableName the table
+ * @param includeTableSnapshotPath true if return the paths of table snapshots
+ * @return the path list
+ * @throws IOException if an error occurred
+ */
+ private List getDefaultTablePath(TableName tableName, boolean includeTableSnapshotPath)
+ throws IOException {
+ List paths = Lists.newArrayList(pathHelper.getTmpTableDir(tableName),
+ pathHelper.getTableDir(tableName), pathHelper.getArchiveTableDir(tableName));
+ if (includeTableSnapshotPath) {
+ paths.addAll(getTableSnapshots(tableName).stream()
+ .map(snap -> pathHelper.getSnapshotDir(snap)).collect(Collectors.toList()));
+ }
+ return paths;
+ }
+
+ /**
+ * Return users with namespace read permission
+ * @param namespace the namespace
+ * @return users with namespace read permission
+ * @throws IOException if an error occurred
+ */
+ private Set getUsersWithNamespaceReadAction(String namespace) throws IOException {
+ return PermissionStorage.getNamespacePermissions(conf, namespace).entries().stream()
+ .filter(entry -> entry.getValue().getPermission().implies(READ))
+ .map(entry -> entry.getKey()).collect(Collectors.toSet());
+ }
+
+ /**
+ * Return users with table read permission
+ * @param tableName the table
+ * @return users with table read permission
+ * @throws IOException if an error occurred
+ */
+ private Set getUsersWithTableReadAction(TableName tableName) throws IOException {
+ return PermissionStorage.getTablePermissions(conf, tableName).entries().stream()
+ .filter(entry -> entry.getValue().getPermission().implies(READ))
+ .map(entry -> entry.getKey()).collect(Collectors.toSet());
+ }
+
+ private List getNamespaceSnapshots(String namespace) throws IOException {
+ return getSnapshots((snapDesc) -> TableName.valueOf(snapDesc.getTable()).getNamespaceAsString()
+ .equals(namespace));
+ }
+
+ private List getTableSnapshots(TableName tableName) throws IOException {
+ return getSnapshots((snapDesc) -> TableName.valueOf(snapDesc.getTable()) == tableName);
+ }
+
+ private List getSnapshots(Predicate predicate)
+ throws IOException {
+ return masterServices.getSnapshotManager().getCompletedSnapshots().stream()
+ .filter(snapshotDescription -> predicate.test(snapshotDescription))
+ .map(snapshotDescription -> snapshotDescription.getName()).collect(Collectors.toList());
+ }
+
+ protected PathHelper getPathHelper() {
+ return pathHelper;
+ }
+
+ /**
+ * Set HDFS acls
+ * @param hdfsAclOperations the {@link HDFSAclOperation} list
+ */
+ private void setHDFSAcl(List hdfsAclOperations)
+ throws InterruptedException, ExecutionException {
+ for (HDFSAclOperation hdfsAclOperation : hdfsAclOperations) {
+ setSingleHDFSAcl(hdfsAclOperation).get();
+ }
+ }
+
+ private CompletableFuture setSingleHDFSAcl(HDFSAclOperation acl) {
+ return CompletableFuture.supplyAsync(() -> {
+ try {
+ acl.setAcl();
+ } catch (IOException e) {
+ LOG.error("Set HDFS acl error for path {}", acl.path.toString(), e);
+ }
+ return acl;
+ }, pool).thenComposeAsync(acl2 -> {
+ List childAclOperations = null;
+ try {
+ childAclOperations = acl2.getChildAclOperations();
+ } catch (IOException e) {
+ LOG.error("Set HDFS acl error for path {}", acl2.path.toString(), e);
+ }
+ return setHDFSAclParallel(childAclOperations);
+ }, pool);
+ }
+
+ private CompletableFuture setHDFSAclParallel(List operations) {
+ List> futures = operations.stream()
+ .map(operation -> setSingleHDFSAcl(operation)).collect(Collectors.toList());
+ CompletableFuture future =
+ CompletableFuture.allOf(futures.toArray(new CompletableFuture[futures.size()]));
+ return future;
+ }
+
+ /**
+ * Inner class used to describe modify or remove what acl entries for files or directories(and
+ * child files)
+ */
+ private static class HDFSAclOperation {
+ enum OperationType {
+ MODIFY, REMOVE
+ }
+
+ interface Operation {
+ void apply(FileSystem fs, Path path, List aclList) throws IOException;
+ }
+
+ private FileSystem fs;
+ private Path path;
+ private List dirAcl;
+ private List fileAcl;
+ private boolean recursive;
+ private Operation operation;
+ private List skipPaths;
+
+ HDFSAclOperation(FileSystem fs, Path path, Set users, OperationType operationType,
+ FsAction fsAction, boolean recursive, List skipPaths) {
+ this.fs = fs;
+ this.path = path;
+ this.dirAcl = getDefaultAclEntries(users, fsAction);
+ this.fileAcl = getAccessAclEntries(users, fsAction);
+ this.recursive = recursive;
+ this.skipPaths = skipPaths;
+ if (operationType == OperationType.MODIFY) {
+ operation =
+ (fileSystem, modifyPath, aclList) -> fileSystem.modifyAclEntries(modifyPath, aclList);
+ } else if (operationType == OperationType.REMOVE) {
+ operation =
+ (fileSystem, modifyPath, aclList) -> fileSystem.removeAclEntries(modifyPath, aclList);
+ }
+ }
+
+ HDFSAclOperation(Path path, HDFSAclOperation parent) {
+ this.fs = parent.fs;
+ this.path = path;
+ this.dirAcl = parent.dirAcl;
+ this.fileAcl = parent.fileAcl;
+ this.operation = parent.operation;
+ this.recursive = parent.recursive;
+ this.skipPaths = parent.skipPaths;
+ }
+
+ List getChildAclOperations() throws IOException {
+ List hdfsAclOperations = new ArrayList<>();
+ if (!skipPaths.contains(path) && recursive && fs.isDirectory(path)) {
+ FileStatus[] fileStatuses = fs.listStatus(path);
+ for (FileStatus fileStatus : fileStatuses) {
+ hdfsAclOperations.add(new HDFSAclOperation(fileStatus.getPath(), this));
+ }
+ }
+ return hdfsAclOperations;
+ }
+
+ void setAcl() throws IOException {
+ if (!skipPaths.contains(path) && fs.exists(path)) {
+ if (fs.isDirectory(path)) {
+ if (recursive) {
+ operation.apply(fs, path, dirAcl);
+ } else {
+ operation.apply(fs, path, fileAcl);
+ }
+ } else {
+ operation.apply(fs, path, fileAcl);
+ }
+ }
+ }
+
+ private List getAccessAclEntries(Set users, FsAction action) {
+ List aclList = new ArrayList<>();
+ for (String user : users) {
+ aclList.add(aclEntry(ACCESS, user, action));
+ }
+ return aclList;
+ }
+
+ private List getDefaultAclEntries(Set users, FsAction action) {
+ List dirAclList = new ArrayList<>();
+ for (String user : users) {
+ dirAclList.add(aclEntry(ACCESS, user, action));
+ dirAclList.add(aclEntry(DEFAULT, user, action));
+ }
+ return dirAclList;
+ }
+
+ private AclEntry aclEntry(AclEntryScope scope, String name, FsAction action) {
+ return new AclEntry.Builder().setScope(scope).setType(USER).setName(name)
+ .setPermission(action).build();
+ }
+ }
+
+ protected static final class PathHelper {
+ Configuration conf;
+ Path rootDir;
+ Path tmpDataDir;
+ Path dataDir;
+ Path archiveDataDir;
+ Path snapshotDir;
+
+ PathHelper(Configuration conf) {
+ this.conf = conf;
+ rootDir = new Path(conf.get(HConstants.HBASE_DIR));
+ tmpDataDir = new Path(new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY),
+ HConstants.BASE_NAMESPACE_DIR);
+ dataDir = new Path(rootDir, HConstants.BASE_NAMESPACE_DIR);
+ archiveDataDir = new Path(new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY),
+ HConstants.BASE_NAMESPACE_DIR);
+ snapshotDir = new Path(rootDir, HConstants.SNAPSHOT_DIR_NAME);
+ }
+
+ Path getRootDir() {
+ return rootDir;
+ }
+
+ Path getDataDir() {
+ return dataDir;
+ }
+
+ Path getTmpDir() {
+ return new Path(rootDir, HConstants.HBASE_TEMP_DIRECTORY);
+ }
+
+ Path getTmpDataDir() {
+ return tmpDataDir;
+ }
+
+ Path getArchiveDir() {
+ return new Path(rootDir, HConstants.HFILE_ARCHIVE_DIRECTORY);
+ }
+
+ Path getArchiveDataDir() {
+ return archiveDataDir;
+ }
+
+ Path getDataNsDir(String namespace) {
+ return new Path(dataDir, namespace);
+ }
+
+ Path getTableDir(TableName tableName) {
+ return new Path(getDataNsDir(tableName.getNamespaceAsString()),
+ tableName.getQualifierAsString());
+ }
+
+ Path getArchiveNsDir(String namespace) {
+ return new Path(archiveDataDir, namespace);
+ }
+
+ Path getArchiveTableDir(TableName tableName) {
+ return new Path(getArchiveNsDir(tableName.getNamespaceAsString()),
+ tableName.getQualifierAsString());
+ }
+
+ Path getTmpNsDir(String namespace) {
+ return new Path(tmpDataDir, namespace);
+ }
+
+ Path getTmpTableDir(TableName tableName) {
+ return new Path(getTmpNsDir(tableName.getNamespaceAsString()),
+ tableName.getQualifierAsString());
+ }
+
+ Path getSnapshotRootDir() {
+ return snapshotDir;
+ }
+
+ Path getSnapshotDir(String snapshot) {
+ return new Path(snapshotDir, snapshot);
+ }
+
+ FileSystem getFileSystem() throws IOException {
+ return rootDir.getFileSystem(conf);
+ }
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
index bcf070a93004..69eee94d25a2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/PermissionStorage.java
@@ -263,6 +263,7 @@ private static void removePermissionRecord(Configuration conf, UserPermission us
static void removeTablePermissions(Configuration conf, TableName tableName, Table t)
throws IOException{
Delete d = new Delete(tableName.getName());
+ d.addFamily(ACL_LIST_FAMILY);
if (LOG.isDebugEnabled()) {
LOG.debug("Removing permissions of removed table "+ tableName);
@@ -280,7 +281,7 @@ static void removeTablePermissions(Configuration conf, TableName tableName, Tabl
static void removeNamespacePermissions(Configuration conf, String namespace, Table t)
throws IOException{
Delete d = new Delete(Bytes.toBytes(toNamespaceEntry(namespace)));
-
+ d.addFamily(ACL_LIST_FAMILY);
if (LOG.isDebugEnabled()) {
LOG.debug("Removing permissions of removed namespace "+ namespace);
}
@@ -839,11 +840,11 @@ public static ListMultimap readPermissions(byte[] data,
}
public static boolean isGlobalEntry(byte[] entryName) {
- return entryName != null && TableName.valueOf(entryName).equals(ACL_TABLE_NAME);
+ return !isNamespaceEntry(entryName) && TableName.valueOf(entryName).equals(ACL_TABLE_NAME);
}
public static boolean isNamespaceEntry(String entryName) {
- return entryName != null && entryName.charAt(0) == NAMESPACE_PREFIX;
+ return entryName != null && entryName.length() > 0 && entryName.charAt(0) == NAMESPACE_PREFIX;
}
public static boolean isNamespaceEntry(byte[] entryName) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclController.java
new file mode 100644
index 000000000000..4db861d17667
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestHDFSAclController.java
@@ -0,0 +1,642 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.security.access;
+
+import static org.apache.hadoop.hbase.security.access.Permission.Action.READ;
+import static org.apache.hadoop.hbase.security.access.Permission.Action.WRITE;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.client.TableDescriptor;
+import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
+import org.apache.hadoop.hbase.client.TableSnapshotScanner;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.testclassification.LargeTests;
+import org.apache.hadoop.hbase.testclassification.SecurityTests;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Category({ SecurityTests.class, LargeTests.class })
+public class TestHDFSAclController {
+ @ClassRule
+ public static final HBaseClassTestRule CLASS_RULE =
+ HBaseClassTestRule.forClass(TestHDFSAclController.class);
+ @Rule
+ public TestName name = new TestName();
+ private static final Logger LOG = LoggerFactory.getLogger(TestHDFSAclController.class);
+
+ private static final String UN_GRANT_USER = "un_grant_user";
+ private static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+ private static Configuration conf = TEST_UTIL.getConfiguration();
+ private static Admin admin = null;
+ private static FileSystem fs = null;
+ private static Path rootDir = null;
+ private static User unGrantUser = null;
+
+ @BeforeClass
+ public static void setupBeforeClass() throws Exception {
+ // enable hdfs acl and set umask to 027
+ conf.setBoolean("dfs.namenode.acls.enabled", true);
+ conf.set("fs.permissions.umask-mode", "027");
+ // enable hbase hdfs acl feature
+ conf.setBoolean(HDFSAclController.HDFS_ACL_ENABLE, true);
+ // enable secure
+ conf.set(User.HBASE_SECURITY_CONF_KEY, "simple");
+ conf.set(HDFSAclController.SNAPSHOT_RESTORE_TMP_DIR, HDFSAclController.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+ SecureTestUtil.enableSecurity(conf);
+ // add HDFSAclController coprocessor
+ conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
+ conf.get(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY) + ","
+ + HDFSAclController.class.getName());
+
+ TEST_UTIL.startMiniCluster();
+ admin = TEST_UTIL.getAdmin();
+ rootDir = TEST_UTIL.getDefaultRootDirPath();
+ fs = rootDir.getFileSystem(conf);
+ unGrantUser = User.createUserForTesting(conf, UN_GRANT_USER, new String[] {});
+
+ // set hbase directory permission
+ Path path = rootDir;
+ while (path != null) {
+ fs.setPermission(path, HDFSAclController.ACL_ENABLE_PUBLIC_HFILE_PERMISSION);
+ path = path.getParent();
+ }
+ // set restore directory permission
+ Path restoreDir = new Path(HDFSAclController.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+ if (!fs.exists(restoreDir)) {
+ fs.mkdirs(restoreDir);
+ fs.setPermission(restoreDir, HDFSAclController.ACL_ENABLE_RESTORE_HFILE_PERMISSION);
+ }
+ path = restoreDir.getParent();
+ while (path != null) {
+ fs.setPermission(path, HDFSAclController.ACL_ENABLE_PUBLIC_HFILE_PERMISSION);
+ path = path.getParent();
+ }
+
+ TEST_UTIL.waitTableAvailable(PermissionStorage.ACL_TABLE_NAME);
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() throws Exception {
+ TEST_UTIL.shutdownMiniCluster();
+ }
+
+ @Test
+ public void testGrantGlobal() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace1 = name.getMethodName();
+ String namespace2 = namespace1 + "2";
+ String namespace3 = namespace1 + "3";
+ TableName table1 = TableName.valueOf(namespace1, "t1");
+ TableName table12 = TableName.valueOf(namespace1, "t2");
+ TableName table21 = TableName.valueOf(namespace2, "t21");
+ TableName table3 = TableName.valueOf(namespace3, "t3");
+ TableName table31 = TableName.valueOf(namespace3, "t31");
+ String snapshot = namespace1 + "t1";
+ String snapshot2 = namespace1 + "t2";
+ String snapshot21 = namespace2 + "t21";
+ String snapshot3 = namespace1 + "t3";
+ String snapshot31 = namespace1 + "t31";
+
+ createNamespace(namespace1);
+ createTableAndPut(table1);
+ admin.snapshot(snapshot, table1);
+
+ // case 1: grant G(R) -> grant G(W) -> grant G(R)
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ admin.grant(
+ new UserPermission(grantUserName, Permission.newBuilder().withActions(WRITE).build()), true);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, WRITE);
+ canUserScanSnapshot(grantUser, snapshot, -1);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+
+ // case 2: grant G(R),N(R) -> G(W)
+ admin.grant(new UserPermission(grantUserName,
+ Permission.newBuilder(namespace1).withActions(READ).build()),
+ false);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, WRITE);
+ // table in ns1
+ createTableAndPut(table12);
+ admin.snapshot(snapshot2, table12);
+ // table in ns2
+ createNamespace(namespace2);
+ createTableAndPut(table21);
+ admin.snapshot(snapshot21, table21);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 6);
+ canUserScanSnapshot(grantUser, snapshot21, -1);
+
+ // case 3: grant G(R),T(R) -> G(W)
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ createNamespace(namespace3);
+ createTableAndPut(table3);
+ admin.snapshot(snapshot3, table3);
+ grantOnTable(grantUserName, table3, READ);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, WRITE);
+ createTableAndPut(table31);
+ admin.snapshot(snapshot31, table31);
+ canUserScanSnapshot(grantUser, snapshot3, 6);
+ canUserScanSnapshot(grantUser, snapshot31, -1);
+ }
+
+ @Test
+ public void testGrantNamespace() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName table = TableName.valueOf(namespace, "t1");
+ TableName table2 = TableName.valueOf(namespace, "t2");
+ TableName table3 = TableName.valueOf(namespace, "t3");
+ String snapshot = namespace + "t1";
+ String snapshot2 = namespace + "t2";
+ String snapshot3 = namespace + "t3";
+
+ createNamespace(namespace);
+ createTableAndPut(table);
+ admin.snapshot(snapshot, table);
+
+ // case 1: grant N(R) -> grant N(W) -> grant N(R)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ createTableAndPut(table3);
+ admin.snapshot(snapshot3, table3);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot3, 6);
+ canUserScanSnapshot(unGrantUser, snapshot, -1);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, WRITE);
+ canUserScanSnapshot(grantUser, snapshot, -1);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+
+ // case 2: grant T(R) -> N(W)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ grantOnTable(grantUserName, table, READ);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, WRITE);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ createTableAndPut(table2);
+ admin.snapshot(snapshot2, table2);
+ canUserScanSnapshot(grantUser, snapshot2, -1);
+ grantOnTable(grantUserName, table, WRITE);
+
+ // case 3: grant G(R) -> N(W)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, WRITE);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 6);
+ canUserScanSnapshot(grantUser, snapshot3, 6);
+ }
+
+ @Test
+ public void testGrantTable() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName table = TableName.valueOf(namespace, "t1");
+ TableName table2 = TableName.valueOf(namespace, "t2");
+ String snapshot = namespace + "t1";
+ String snapshot2 = namespace + "t1-2";
+ String snapshot3 = namespace + "t2";
+
+ createNamespace(namespace);
+ try (Table t = createTable(table)) {
+ put(t);
+ admin.snapshot(snapshot, table);
+ // table owner can scan table snapshot
+ canUserScanSnapshot(User.createUserForTesting(conf, "owner", new String[] {}), snapshot, 6);
+ // case 1: grant table family(R)
+ SecureTestUtil.grantOnTable(TEST_UTIL, grantUserName, table, COLUMN1, null, READ);
+ canUserScanSnapshot(grantUser, snapshot, -1);
+ // case 2: grant T(R)
+ grantOnTable(grantUserName, table, READ);
+ put2(t);
+ admin.snapshot(snapshot2, table);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 10);
+ }
+ // create t2 and snapshot
+ createTableAndPut(table2);
+ admin.snapshot(snapshot3, table2);
+ canUserScanSnapshot(grantUser, snapshot3, -1);
+
+ // case 3: grant T(R) -> grant T(W)
+ grantOnTable(grantUserName, table, WRITE);
+ canUserScanSnapshot(grantUser, snapshot, -1);
+ canUserScanSnapshot(grantUser, snapshot2, -1);
+ }
+
+ @Test
+ public void testRevokeGlobal() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName table1 = TableName.valueOf(namespace, "t1");
+ TableName table2 = TableName.valueOf(namespace, "t2");
+ TableName table3 = TableName.valueOf(namespace, "t3");
+ String snapshot1 = namespace + "t1";
+ String snapshot2 = namespace + "t2";
+ String snapshot3 = namespace + "t3";
+
+ createNamespace(namespace);
+ createTableAndPut(table1);
+ admin.snapshot(snapshot1, table1);
+ // case 1: grant G(R) -> revoke G(R)
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot1, -1);
+
+ // case 2: grant G(R), grant N(R), grant T(R) -> revoke G(R)
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ grantOnTable(grantUserName, table1, READ);
+ SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot1, 6);
+ createTableAndPut(table2);
+ admin.snapshot(snapshot2, table2);
+ canUserScanSnapshot(grantUser, snapshot2, 6);
+ SecureTestUtil.revokeFromNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ canUserScanSnapshot(grantUser, snapshot2, -1);
+
+ // case 3: grant G(R), grant T(R) -> revoke G(R)
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot1, 6);
+ canUserScanSnapshot(grantUser, snapshot2, -1);
+ createTableAndPut(table3);
+ admin.snapshot(snapshot3, table3);
+ canUserScanSnapshot(grantUser, snapshot3, -1);
+ }
+
+ @Test
+ public void testRevokeNamespace() throws Exception {
+ String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName table1 = TableName.valueOf(namespace, "t1");
+ TableName table2 = TableName.valueOf(namespace, "t2");
+ TableName table3 = TableName.valueOf(namespace, "t3");
+ TableName table4 = TableName.valueOf(namespace, "t4");
+ String snapshot1 = namespace + "t1";
+ String snapshot2 = namespace + "t2";
+ String snapshot3 = namespace + "t3";
+ String snapshot4 = namespace + "t4";
+
+ createNamespace(namespace);
+ createTableAndPut(table1);
+ admin.snapshot(snapshot1, table1);
+
+ // case 1: grant N(R) -> revoke N(R)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(namespace).build()));
+ createTableAndPut(table3);
+ admin.snapshot(snapshot3, table3);
+ canUserScanSnapshot(grantUser, snapshot1, -1);
+ canUserScanSnapshot(grantUser, snapshot3, -1);
+
+ // case 2: grant N(R), grant G(R) -> revoke N(R)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(namespace).build()));
+ createTableAndPut(table4);
+ admin.snapshot(snapshot4, table4);
+ canUserScanSnapshot(grantUser, snapshot1, 6);
+ canUserScanSnapshot(grantUser, snapshot4, 6);
+ SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
+
+ // case 3: grant N(R), grant T(R) -> revoke N(R)
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ grantOnTable(grantUserName, table1, READ);
+ SecureTestUtil.revokeFromNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ canUserScanSnapshot(grantUser, snapshot1, 6);
+ createTable(table2);
+ admin.snapshot(snapshot2, table2);
+ canUserScanSnapshot(grantUser, snapshot2, -1);
+ }
+
+ @Test
+ public void testRevokeTable() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName table = TableName.valueOf(namespace, "t1");
+ String snapshot = namespace + "t1";
+
+ createNamespace(namespace);
+ createTableAndPut(table);
+ admin.snapshot(snapshot, table);
+
+ // case 1: grant T(R) -> revoke table family
+ grantOnTable(grantUserName, table, READ);
+ SecureTestUtil.revokeFromTable(TEST_UTIL, grantUserName, table, COLUMN1, null, READ);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+
+ // case 2: grant T(R) -> revoke T(R)
+ grantOnTable(grantUserName, table, READ);
+ admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
+ canUserScanSnapshot(grantUser, snapshot, -1);
+
+ // case 3: grant T(R), grant N(R) -> revoke T(R)
+ grantOnTable(grantUserName, table, READ);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ SecureTestUtil.revokeFromNamespace(TEST_UTIL, grantUserName, namespace, READ);
+
+ // case 4: grant T(R), grant G(R) -> revoke T(R)
+ grantOnTable(grantUserName, table, READ);
+ SecureTestUtil.grantGlobal(TEST_UTIL, grantUserName, READ);
+ admin.revoke(new UserPermission(grantUserName, Permission.newBuilder(table).build()));
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ SecureTestUtil.revokeGlobal(TEST_UTIL, grantUserName, READ);
+ canUserScanSnapshot(grantUser, snapshot, -1);
+ }
+
+ @Test
+ public void testTruncateTable() throws Exception {
+ String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+ String grantUserName2 = grantUserName + "2";
+ User grantUser2 = User.createUserForTesting(conf, grantUserName2, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName tableName = TableName.valueOf(namespace, "t1");
+ String snapshot = namespace + "t1";
+ String snapshot2 = namespace + "t1-2";
+ createNamespace(namespace);
+ try (Table t = createTable(tableName)) {
+ put(t);
+ // snapshot
+ admin.snapshot(snapshot, tableName);
+ // grant user2 namespace permission
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName2, namespace, READ);
+ // grant user table permission
+ grantOnTable(grantUserName, tableName, READ);
+ // truncate table
+ admin.disableTable(tableName);
+ admin.truncateTable(tableName, true);
+ put2(t);
+ // snapshot
+ admin.snapshot(snapshot2, tableName);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser2, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 9);
+ canUserScanSnapshot(grantUser2, snapshot2, 9);
+ }
+ }
+
+ @Test
+ public void testRestoreSnapshot() throws Exception {
+ final String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+ String namespace = name.getMethodName();
+ TableName table = TableName.valueOf(namespace, "t1");
+ String snapshot = namespace + "t1";
+ String snapshot2 = namespace + "t1-2";
+ String snapshot3 = namespace + "t1-3";
+
+ createNamespace(namespace);
+ try (Table t = createTable(table)) {
+ put(t);
+ // grant t1, snapshot
+ grantOnTable(grantUserName, table, READ);
+ admin.snapshot(snapshot, table);
+ // delete
+ admin.disableTable(table);
+ admin.deleteTable(table);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+
+ // restore snapshot and restore acl
+ admin.restoreSnapshot(snapshot, true, true);
+ put2(t);
+ // snapshot
+ admin.snapshot(snapshot2, table);
+ // delete
+ admin.disableTable(table);
+ admin.deleteTable(table);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 10);
+
+ // restore snapshot and skip restore acl
+ admin.restoreSnapshot(snapshot);
+ admin.snapshot(snapshot3, table);
+
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ canUserScanSnapshot(grantUser, snapshot2, 10);
+ canUserScanSnapshot(grantUser, snapshot3, -1);
+ }
+ }
+
+ @Test
+ public void testDeleteTable() throws Exception {
+ String namespace = name.getMethodName();
+ String grantUserName1 = namespace + "1";
+ String grantUserName2 = namespace + "2";
+ String grantUserName3 = namespace + "3";
+ User grantUser1 = User.createUserForTesting(conf, grantUserName1, new String[] {});
+ User grantUser2 = User.createUserForTesting(conf, grantUserName2, new String[] {});
+ User grantUser3 = User.createUserForTesting(conf, grantUserName3, new String[] {});
+
+ TableName tableName1 = TableName.valueOf(namespace, "t1");
+ TableName tableName2 = TableName.valueOf(namespace, "t2");
+ String snapshot1 = namespace + "t1";
+ String snapshot2 = namespace + "t2";
+ createNamespace(namespace);
+ try (Table t = createTable(tableName1); Table t2 = createTable(tableName2)) {
+ put(t);
+ put(t2);
+ // snapshot
+ admin.snapshot(snapshot1, tableName1);
+ admin.snapshot(snapshot2, tableName2);
+ // grant user table permission
+ grantOnTable(grantUserName1, tableName1, READ);
+ grantOnTable(grantUserName2, tableName2, READ);
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName3, namespace, READ);
+ // delete table
+ admin.disableTable(tableName1);
+ admin.deleteTable(tableName1);
+ // grantUser2 and grantUser3 should have data/ns acl
+ canUserScanSnapshot(grantUser1, snapshot1, 6);
+ canUserScanSnapshot(grantUser2, snapshot2, 6);
+ canUserScanSnapshot(grantUser3, snapshot2, 6);
+ }
+ }
+
+ @Test
+ public void testDeleteNamespace() throws Exception {
+ String grantUserName = name.getMethodName();
+ User grantUser = User.createUserForTesting(conf, grantUserName, new String[] {});
+
+ String namespace = name.getMethodName();
+ TableName tableName = TableName.valueOf(namespace, "t1");
+ String snapshot = namespace + "t1";
+ createNamespace(namespace);
+ try (Table t = createTable(tableName)) {
+ put(t);
+ // snapshot
+ admin.snapshot(snapshot, tableName);
+ // grant user2 namespace permission
+ SecureTestUtil.grantOnNamespace(TEST_UTIL, grantUserName, namespace, READ);
+ // truncate table
+ admin.disableTable(tableName);
+ admin.deleteTable(tableName);
+ // snapshot
+ admin.deleteNamespace(namespace);
+ canUserScanSnapshot(grantUser, snapshot, 6);
+ }
+ }
+
+ private void grantOnTable(String user, TableName tableName, Permission.Action... actions)
+ throws Exception {
+ SecureTestUtil.grantOnTable(TEST_UTIL, user, tableName, null, null, actions);
+ }
+
+ private void createNamespace(String namespace) throws IOException {
+ createNamespace(admin, namespace);
+ }
+
+ private void createTableAndPut(TableName tableNam) throws IOException {
+ try (Table t = createTable(tableNam)) {
+ put(t);
+ }
+ }
+
+ private Table createTable(TableName tableName) throws IOException {
+ return createTable(TEST_UTIL, tableName);
+ }
+
+ /**
+ * Check if user is able to read expected rows from the specific snapshot
+ * @param user the specific user
+ * @param snapshot the snapshot to be scanned
+ * @param expectedRowCount expected row count read from snapshot, -1 if expects
+ * AccessControlException
+ * @throws IOException user scan snapshot error
+ * @throws InterruptedException user scan snapshot error
+ */
+ private void canUserScanSnapshot(User user, String snapshot, int expectedRowCount)
+ throws IOException, InterruptedException {
+ PrivilegedExceptionAction action =
+ getScanSnapshotAction(conf, snapshot, expectedRowCount);
+ user.runAs(action);
+ }
+
+ private PrivilegedExceptionAction getScanSnapshotAction(Configuration conf,
+ String snapshotName, long expectedRowCount) {
+ PrivilegedExceptionAction action = () -> {
+ try {
+ Path restoreDir = new Path(HDFSAclController.SNAPSHOT_RESTORE_TMP_DIR_DEFAULT);
+ Scan scan = new Scan();
+ TableSnapshotScanner scanner =
+ new TableSnapshotScanner(conf, restoreDir, snapshotName, scan);
+ int rowCount = 0;
+ while (true) {
+ Result result = scanner.next();
+ if (result == null) {
+ break;
+ }
+ rowCount++;
+ }
+ scanner.close();
+ assertEquals(expectedRowCount, rowCount);
+ } catch (Exception e) {
+ LOG.error("Scan snapshot error", e);
+ assertEquals(expectedRowCount, -1);
+ }
+ return null;
+ };
+ return action;
+ }
+
+ private void createNamespace(Admin admin, String namespace) throws IOException {
+ NamespaceDescriptor namespaceDescriptor = NamespaceDescriptor.create(namespace).build();
+ admin.createNamespace(namespaceDescriptor);
+ }
+
+ private static final byte[] COLUMN1 = Bytes.toBytes("A");
+ private static final byte[] COLUMN2 = Bytes.toBytes("B");
+
+ private Table createTable(HBaseTestingUtility utility, TableName tableName)
+ throws IOException {
+ TableDescriptor td = TableDescriptorBuilder.newBuilder(tableName)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN1).build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(COLUMN2).build())
+ .setOwner(User.createUserForTesting(conf, "owner", new String[] {})).build();
+ byte[][] splits = new byte[][] { Bytes.toBytes("2"), Bytes.toBytes("4") };
+ return utility.createTable(td, splits);
+ }
+
+ private void put(Table hTable) throws IOException {
+ List puts = new ArrayList<>();
+ for (int i = 0; i < 6; i++) {
+ Put put = new Put(Bytes.toBytes(i));
+ put.addColumn(COLUMN1, null, Bytes.toBytes(i));
+ put.addColumn(COLUMN2, null, Bytes.toBytes(i + 1));
+ puts.add(put);
+ }
+ hTable.put(puts);
+ }
+
+ private void put2(Table hTable) throws IOException {
+ List puts = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ if (i == 5) {
+ continue;
+ }
+ Put put = new Put(Bytes.toBytes(i));
+ put.addColumn(COLUMN1, null, Bytes.toBytes(i + 2));
+ put.addColumn(COLUMN2, null, Bytes.toBytes(i + 3));
+ puts.add(put);
+ }
+ hTable.put(puts);
+ }
+}
|