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: + *

+ *

+ *

+ * The implementation of this feature is as followings: + *

+ *

+ */ +@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); + } +}