http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java new file mode 100644 index 0000000..fb7d246 --- /dev/null +++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/MetastoreAuthzBindingBase.java @@ -0,0 +1,450 @@ +/** + * 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.sentry.binding.metastore; + +import java.io.File; +import java.io.IOException; +import java.net.MalformedURLException; +import java.net.URISyntaxException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import javax.security.auth.login.LoginException; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStorePreEventListener; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent; +import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent; +import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent; +import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent; +import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent; +import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent; +import org.apache.hadoop.hive.metastore.events.PreDropTableEvent; +import org.apache.hadoop.hive.metastore.events.PreEventContext; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.sentry.binding.hive.authz.HiveAuthzBinding; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars; +import org.apache.sentry.core.common.utils.PathUtils; +import org.apache.sentry.core.model.db.AccessURI; +import org.apache.sentry.core.model.db.DBModelAuthorizable; +import org.apache.sentry.core.model.db.Database; +import org.apache.sentry.core.model.db.Server; +import org.apache.sentry.core.model.db.Table; + +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; + +/** + * Sentry binding for Hive Metastore. The binding is integrated into Metastore + * via the pre-event listener which are fired prior to executing the metadata + * action. This point we are only authorizing metadata writes since the listners + * are not fired from read events. Each action builds a input and output + * hierarchy as per the objects used in the given operations. This is then + * passed down to the hive binding which handles the authorization. This ensures + * that we follow the same privilege model and policies. + */ +public abstract class MetastoreAuthzBindingBase extends MetaStorePreEventListener { + + /** + * Build the set of object hierarchies ie fully qualified db model objects + */ + protected static class HierarcyBuilder { + private List<List<DBModelAuthorizable>> authHierarchy; + + public HierarcyBuilder() { + authHierarchy = new ArrayList<List<DBModelAuthorizable>>(); + } + + public HierarcyBuilder addServerToOutput(Server server) { + List<DBModelAuthorizable> serverHierarchy = new ArrayList<DBModelAuthorizable>(); + serverHierarchy.add(server); + authHierarchy.add(serverHierarchy); + return this; + } + + public HierarcyBuilder addDbToOutput(Server server, String dbName) { + List<DBModelAuthorizable> dbHierarchy = new ArrayList<DBModelAuthorizable>(); + addServerToOutput(server); + dbHierarchy.add(server); + dbHierarchy.add(new Database(dbName)); + authHierarchy.add(dbHierarchy); + return this; + } + + public HierarcyBuilder addUriToOutput(Server server, String uriPath, + String warehouseDirPath) throws MetaException { + List<DBModelAuthorizable> uriHierarchy = new ArrayList<DBModelAuthorizable>(); + addServerToOutput(server); + uriHierarchy.add(server); + try { + uriHierarchy.add(new AccessURI(PathUtils.parseDFSURI(warehouseDirPath, + uriPath))); + } catch (URISyntaxException e) { + throw new MetaException("Error paring the URI " + e.getMessage()); + } + authHierarchy.add(uriHierarchy); + return this; + } + + public HierarcyBuilder addTableToOutput(Server server, String dbName, + String tableName) { + List<DBModelAuthorizable> tableHierarchy = new ArrayList<DBModelAuthorizable>(); + addDbToOutput(server, dbName); + tableHierarchy.add(server); + tableHierarchy.add(new Database(dbName)); + tableHierarchy.add(new Table(tableName)); + authHierarchy.add(tableHierarchy); + return this; + } + + public List<List<DBModelAuthorizable>> build() { + return authHierarchy; + } + } + + private HiveAuthzConf authzConf; + private final Server authServer; + private final HiveConf hiveConf; + private final ImmutableSet<String> serviceUsers; + private HiveAuthzBinding hiveAuthzBinding; + private final String warehouseDir; + protected static boolean sentryCacheOutOfSync = false; + + public MetastoreAuthzBindingBase(Configuration config) throws Exception { + super(config); + String hiveAuthzConf = config.get(HiveAuthzConf.HIVE_SENTRY_CONF_URL); + if (hiveAuthzConf == null + || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) { + throw new IllegalArgumentException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf + + "' is invalid."); + } + try { + authzConf = new HiveAuthzConf(new URL(hiveAuthzConf)); + } catch (MalformedURLException e) { + throw new IllegalArgumentException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " specifies a malformed URL '" + + hiveAuthzConf + "'", e); + } + hiveConf = new HiveConf(config, this.getClass()); + this.authServer = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME + .getVar())); + serviceUsers = ImmutableSet.copyOf(toTrimedLower(Sets.newHashSet(authzConf + .getStrings(AuthzConfVars.AUTHZ_METASTORE_SERVICE_USERS.getVar(), + new String[] { "" })))); + warehouseDir = hiveConf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE); + + } + + /** + * Main listener callback which is the entry point for Sentry + */ + @Override + public void onEvent(PreEventContext context) throws MetaException, + NoSuchObjectException, InvalidOperationException { + + if (!needsAuthorization(getUserName())) { + return; + } + switch (context.getEventType()) { + case CREATE_TABLE: + authorizeCreateTable((PreCreateTableEvent) context); + break; + case DROP_TABLE: + authorizeDropTable((PreDropTableEvent) context); + break; + case ALTER_TABLE: + authorizeAlterTable((PreAlterTableEvent) context); + break; + case ADD_PARTITION: + authorizeAddPartition((PreAddPartitionEvent) context); + break; + case DROP_PARTITION: + authorizeDropPartition((PreDropPartitionEvent) context); + break; + case ALTER_PARTITION: + authorizeAlterPartition((PreAlterPartitionEvent) context); + break; + case CREATE_DATABASE: + authorizeCreateDatabase(); + break; + case DROP_DATABASE: + authorizeDropDatabase((PreDropDatabaseEvent) context); + break; + case LOAD_PARTITION_DONE: + // noop for now + break; + default: + break; + } + } + + private void authorizeCreateDatabase() + throws InvalidOperationException, MetaException { + authorizeMetastoreAccess(HiveOperation.CREATEDATABASE, + new HierarcyBuilder().addServerToOutput(getAuthServer()).build(), + new HierarcyBuilder().addServerToOutput(getAuthServer()).build()); + } + + private void authorizeDropDatabase(PreDropDatabaseEvent context) + throws InvalidOperationException, MetaException { + authorizeMetastoreAccess(HiveOperation.DROPDATABASE, + new HierarcyBuilder() +.addDbToOutput(getAuthServer(), + context.getDatabase().getName()).build(), + new HierarcyBuilder().addDbToOutput(getAuthServer(), + context.getDatabase().getName()).build()); + } + + private void authorizeCreateTable(PreCreateTableEvent context) + throws InvalidOperationException, MetaException { + HierarcyBuilder inputBuilder = new HierarcyBuilder(); + inputBuilder.addDbToOutput(getAuthServer(), context.getTable().getDbName()); + HierarcyBuilder outputBuilder = new HierarcyBuilder(); + outputBuilder.addDbToOutput(getAuthServer(), context.getTable().getDbName()); + + if (!StringUtils.isEmpty(context.getTable().getSd().getLocation())) { + String uriPath; + try { + uriPath = PathUtils.parseDFSURI(warehouseDir, + getSdLocation(context.getTable().getSd())); + } catch(URISyntaxException e) { + throw new MetaException(e.getMessage()); + } + inputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir); + } + authorizeMetastoreAccess(HiveOperation.CREATETABLE, inputBuilder.build(), + outputBuilder.build()); + } + + private void authorizeDropTable(PreDropTableEvent context) + throws InvalidOperationException, MetaException { + authorizeMetastoreAccess( + HiveOperation.DROPTABLE, + new HierarcyBuilder().addTableToOutput(getAuthServer(), + context.getTable().getDbName(), context.getTable().getTableName()) + .build(), + new HierarcyBuilder().addTableToOutput(getAuthServer(), + context.getTable().getDbName(), context.getTable().getTableName()) + .build()); + } + + private void authorizeAlterTable(PreAlterTableEvent context) + throws InvalidOperationException, MetaException { + /* + * There are multiple alter table options and it's tricky to figure which is + * attempted here. Currently all alter table needs full level privilege + * except the for setting location which also needs a privile on URI. Hence + * we set initially set the operation to ALTERTABLE_ADDCOLS. If the client + * has specified the location, then change to ALTERTABLE_LOCATION + */ + HiveOperation operation = HiveOperation.ALTERTABLE_ADDCOLS; + HierarcyBuilder inputBuilder = new HierarcyBuilder(); + inputBuilder.addTableToOutput(getAuthServer(), context.getOldTable() + .getDbName(), context.getOldTable().getTableName()); + HierarcyBuilder outputBuilder = new HierarcyBuilder(); + outputBuilder.addTableToOutput(getAuthServer(), context.getOldTable() + .getDbName(), context.getOldTable().getTableName()); + + // if the operation requires location change, then add URI privilege check + String oldLocationUri; + String newLocationUri; + try { + oldLocationUri = PathUtils.parseDFSURI(warehouseDir, + getSdLocation(context.getOldTable().getSd())); + newLocationUri = PathUtils.parseDFSURI(warehouseDir, + getSdLocation(context.getNewTable().getSd())); + } catch (URISyntaxException e) { + throw new MetaException(e.getMessage()); + } + if (oldLocationUri.compareTo(newLocationUri) != 0) { + outputBuilder.addUriToOutput(getAuthServer(), newLocationUri, + warehouseDir); + operation = HiveOperation.ALTERTABLE_LOCATION; + } + authorizeMetastoreAccess( + operation, + inputBuilder.build(), outputBuilder.build()); + + } + + private void authorizeAddPartition(PreAddPartitionEvent context) + throws InvalidOperationException, MetaException, NoSuchObjectException { + for (Partition mapiPart : context.getPartitions()) { + HierarcyBuilder inputBuilder = new HierarcyBuilder(); + inputBuilder.addTableToOutput(getAuthServer(), mapiPart + .getDbName(), mapiPart.getTableName()); + HierarcyBuilder outputBuilder = new HierarcyBuilder(); + outputBuilder.addTableToOutput(getAuthServer(), mapiPart + .getDbName(), mapiPart.getTableName()); + // check if we need to validate URI permissions when storage location is + // non-default, ie something not under the parent table + + String partitionLocation = null; + if (mapiPart.isSetSd()) { + partitionLocation = mapiPart.getSd().getLocation(); + } + if (!StringUtils.isEmpty(partitionLocation)) { + String tableLocation = context + .getHandler() + .get_table(mapiPart.getDbName(), + mapiPart.getTableName()).getSd().getLocation(); + String uriPath; + try { + uriPath = PathUtils.parseDFSURI(warehouseDir, mapiPart + .getSd().getLocation()); + } catch (URISyntaxException e) { + throw new MetaException(e.getMessage()); + } + if (!partitionLocation.equals(tableLocation) && + !partitionLocation.startsWith(tableLocation + File.separator)) { + outputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir); + } + } + authorizeMetastoreAccess(HiveOperation.ALTERTABLE_ADDPARTS, + inputBuilder.build(), outputBuilder.build()); + } + } + + protected void authorizeDropPartition(PreDropPartitionEvent context) + throws InvalidOperationException, MetaException { + authorizeMetastoreAccess( + HiveOperation.ALTERTABLE_DROPPARTS, + new HierarcyBuilder().addTableToOutput(getAuthServer(), + context.getPartition().getDbName(), + context.getPartition().getTableName()).build(), + new HierarcyBuilder().addTableToOutput(getAuthServer(), + context.getPartition().getDbName(), + context.getPartition().getTableName()).build()); + } + + private void authorizeAlterPartition(PreAlterPartitionEvent context) + throws InvalidOperationException, MetaException, NoSuchObjectException { + /* + * There are multiple alter partition options and it's tricky to figure out + * which is attempted here. Currently all alter partition need full level + * privilege except the for setting location which also needs a privilege on + * URI. Currently we don't try to distinguish the operation type. All alter + * partitions are treated as set-location + */ + HierarcyBuilder inputBuilder = new HierarcyBuilder().addTableToOutput( + getAuthServer(), context.getDbName(), context.getTableName()); + HierarcyBuilder outputBuilder = new HierarcyBuilder().addTableToOutput( + getAuthServer(), context.getDbName(), context.getTableName()); + + Partition partition = context.getNewPartition(); + String partitionLocation = getSdLocation(partition.getSd()); + if (!StringUtils.isEmpty(partitionLocation)) { + String tableLocation = context.getHandler().get_table( + partition.getDbName(), partition.getTableName()).getSd().getLocation(); + + String uriPath; + try { + uriPath = PathUtils.parseDFSURI(warehouseDir, partitionLocation); + } catch (URISyntaxException e) { + throw new MetaException(e.getMessage()); + } + if (!partitionLocation.startsWith(tableLocation + File.separator)) { + outputBuilder.addUriToOutput(getAuthServer(), uriPath, warehouseDir); + } + } + authorizeMetastoreAccess( + HiveOperation.ALTERPARTITION_LOCATION, + inputBuilder.build(), outputBuilder.build()); + } + + protected InvalidOperationException invalidOperationException(Exception e) { + InvalidOperationException ex = new InvalidOperationException(e.getMessage()); + ex.initCause(e.getCause()); + return ex; + } + + /** + * Assemble the required privileges and requested privileges. Validate using + * Hive bind auth provider + * @param hiveOp + * @param inputHierarchy + * @param outputHierarchy + * @throws InvalidOperationException + */ + protected abstract void authorizeMetastoreAccess(HiveOperation hiveOp, + List<List<DBModelAuthorizable>> inputHierarchy, + List<List<DBModelAuthorizable>> outputHierarchy) + throws InvalidOperationException; + + public Server getAuthServer() { + return authServer; + } + + private boolean needsAuthorization(String userName) { + return !serviceUsers.contains(userName); + } + + private static Set<String> toTrimedLower(Set<String> s) { + Set<String> result = Sets.newHashSet(); + for (String v : s) { + result.add(v.trim().toLowerCase()); + } + return result; + } + + protected HiveAuthzBinding getHiveAuthzBinding() throws Exception { + if (hiveAuthzBinding == null) { + hiveAuthzBinding = new HiveAuthzBinding(HiveAuthzBinding.HiveHook.HiveMetaStore, hiveConf, authzConf); + } + return hiveAuthzBinding; + } + + protected String getUserName() throws MetaException { + try { + return Utils.getUGI().getShortUserName(); + } catch (LoginException e) { + throw new MetaException("Failed to get username " + e.getMessage()); + } catch (IOException e) { + throw new MetaException("Failed to get username " + e.getMessage()); + } + } + + private String getSdLocation(StorageDescriptor sd) { + if (sd == null) { + return ""; + } else { + return sd.getLocation(); + } + } + + public static boolean isSentryCacheOutOfSync() { + return sentryCacheOutOfSync; + } + + public static void setSentryCacheOutOfSync(boolean sentryCacheOutOfSync) { + MetastoreAuthzBindingBase.sentryCacheOutOfSync = sentryCacheOutOfSync; + } + +}
http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java new file mode 100644 index 0000000..b5df287 --- /dev/null +++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryHiveMetaStoreClient.java @@ -0,0 +1,161 @@ +/** + * 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.sentry.binding.metastore; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaHookLoader; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase; +import org.apache.sentry.binding.hive.authz.HiveAuthzBinding; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf; +import org.apache.thrift.TException; + +public class SentryHiveMetaStoreClient extends HiveMetaStoreClient implements + IMetaStoreClient { + + private HiveAuthzBinding hiveAuthzBinding; + private HiveAuthzConf authzConf; + + public SentryHiveMetaStoreClient(HiveConf conf) throws MetaException { + super(conf); + } + + public SentryHiveMetaStoreClient(HiveConf conf, HiveMetaHookLoader hookLoader) + throws MetaException { + super(conf, hookLoader); + } + + @Override + public List<String> getDatabases(String databasePattern) throws MetaException { + return filterDatabases(super.getDatabases(databasePattern)); + } + + @Override + public List<String> getAllDatabases() throws MetaException { + return filterDatabases(super.getAllDatabases()); + } + + @Override + public List<String> getTables(String dbName, String tablePattern) + throws MetaException { + return filterTables(dbName, super.getTables(dbName, tablePattern)); + } + + @Override + public List<String> getAllTables(String dbName) throws MetaException { + return filterTables(dbName, super.getAllTables(dbName)); + } + + @Override + public List<String> listTableNamesByFilter(String dbName, String filter, + short maxTables) throws InvalidOperationException, UnknownDBException, + TException { + return filterTables(dbName, + super.listTableNamesByFilter(dbName, filter, maxTables)); + } + + /** + * Invoke Hive database filtering that removes the entries which use has no + * privileges to access + * + * @param dbList + * @return + * @throws MetaException + */ + private List<String> filterDatabases(List<String> dbList) + throws MetaException { + try { + return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(), + dbList, HiveOperation.SHOWDATABASES, getUserName()); + } catch (SemanticException e) { + throw new MetaException("Error getting DB list " + e.getMessage()); + } + } + + /** + * Invoke Hive table filtering that removes the entries which use has no + * privileges to access + * + * @param dbList + * @return + * @throws MetaException + */ + private List<String> filterTables(String dbName, List<String> tabList) + throws MetaException { + try { + return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(), + tabList, HiveOperation.SHOWTABLES, getUserName(), dbName); + } catch (SemanticException e) { + throw new MetaException("Error getting Table list " + e.getMessage()); + } + } + + private String getUserName() { + return getConf().get(HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME); + } + + /** + * load Hive auth provider + * + * @return + * @throws MetaException + */ + private HiveAuthzBinding getHiveAuthzBinding() throws MetaException { + if (hiveAuthzBinding == null) { + String hiveAuthzConf = getConf().get(HiveAuthzConf.HIVE_SENTRY_CONF_URL); + if (hiveAuthzConf == null + || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) { + throw new MetaException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf + + "' is invalid."); + } + try { + authzConf = new HiveAuthzConf(new URL(hiveAuthzConf)); + } catch (MalformedURLException e) { + throw new MetaException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + + " specifies a malformed URL '" + hiveAuthzConf + "' " + + e.getMessage()); + } + try { + hiveAuthzBinding = new HiveAuthzBinding( + HiveAuthzBinding.HiveHook.HiveMetaStore, getConf(), authzConf); + } catch (Exception e) { + throw new MetaException("Failed to load Hive binding " + e.getMessage()); + } + } + return hiveAuthzBinding; + } + + private HiveConf getConf() { + return SessionState.get().getConf(); + } + +} http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java new file mode 100644 index 0000000..2a0a5b8 --- /dev/null +++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetaStoreFilterHook.java @@ -0,0 +1,201 @@ +/** + * 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.sentry.binding.metastore; + +import java.net.MalformedURLException; +import java.net.URL; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreFilterHook; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.Index; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PartitionSpec; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase; +import org.apache.sentry.binding.hive.authz.HiveAuthzBinding; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf; + +public class SentryMetaStoreFilterHook implements MetaStoreFilterHook { + + static final protected Log LOG = LogFactory.getLog(SentryMetaStoreFilterHook.class); + + private HiveAuthzBinding hiveAuthzBinding; + private HiveAuthzConf authzConf; + + public SentryMetaStoreFilterHook(HiveConf hiveConf) { //NOPMD + } + + @Override + public List<String> filterDatabases(List<String> dbList) { + return filterDb(dbList); + } + + @Override + public Database filterDatabase(Database dataBase) + throws NoSuchObjectException { + return dataBase; + } + + @Override + public List<String> filterTableNames(String dbName, List<String> tableList) { + return filterTab(dbName, tableList); + } + + @Override + public Table filterTable(Table table) throws NoSuchObjectException { + return table; + } + + @Override + public List<Table> filterTables(List<Table> tableList) { + return tableList; + } + + @Override + public List<Partition> filterPartitions(List<Partition> partitionList) { + return partitionList; + } + + @Override + public List<PartitionSpec> filterPartitionSpecs( + List<PartitionSpec> partitionSpecList) { + return partitionSpecList; + } + + @Override + public Partition filterPartition(Partition partition) + throws NoSuchObjectException { + return partition; + } + + @Override + public List<String> filterPartitionNames(String dbName, String tblName, + List<String> partitionNames) { + return partitionNames; + } + + @Override + public Index filterIndex(Index index) throws NoSuchObjectException { + return index; + } + + @Override + public List<String> filterIndexNames(String dbName, String tblName, + List<String> indexList) { + return indexList; + } + + @Override + public List<Index> filterIndexes(List<Index> indexeList) { + return indexeList; + } + + /** + * Invoke Hive database filtering that removes the entries which use has no + * privileges to access + * @param dbList + * @return + * @throws MetaException + */ + private List<String> filterDb(List<String> dbList) { + try { + return HiveAuthzBindingHookBase.filterShowDatabases(getHiveAuthzBinding(), + dbList, HiveOperation.SHOWDATABASES, getUserName()); + } catch (Exception e) { + LOG.warn("Error getting DB list ", e); + return new ArrayList<String>(); + } finally { + close(); + } + } + + /** + * Invoke Hive table filtering that removes the entries which use has no + * privileges to access + * @param tabList + * @return + * @throws MetaException + */ + private List<String> filterTab(String dbName, List<String> tabList) { + try { + return HiveAuthzBindingHookBase.filterShowTables(getHiveAuthzBinding(), + tabList, HiveOperation.SHOWTABLES, getUserName(), dbName); + } catch (Exception e) { + LOG.warn("Error getting Table list ", e); + return new ArrayList<String>(); + } finally { + close(); + } + } + + private String getUserName() { + return getConf().get(HiveAuthzConf.HIVE_SENTRY_SUBJECT_NAME); + } + + /** + * load Hive auth provider + * @return + * @throws MetaException + */ + private HiveAuthzBinding getHiveAuthzBinding() throws MetaException { + if (hiveAuthzBinding == null) { + String hiveAuthzConf = getConf().get(HiveAuthzConf.HIVE_SENTRY_CONF_URL); + if (hiveAuthzConf == null + || (hiveAuthzConf = hiveAuthzConf.trim()).isEmpty()) { + throw new MetaException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + " value '" + hiveAuthzConf + + "' is invalid."); + } + try { + authzConf = new HiveAuthzConf(new URL(hiveAuthzConf)); + } catch (MalformedURLException e) { + throw new MetaException("Configuration key " + + HiveAuthzConf.HIVE_SENTRY_CONF_URL + + " specifies a malformed URL '" + hiveAuthzConf + "' " + + e.getMessage()); + } + try { + hiveAuthzBinding = new HiveAuthzBinding( + HiveAuthzBinding.HiveHook.HiveMetaStore, getConf(), authzConf); + } catch (Exception e) { + throw new MetaException("Failed to load Hive binding " + e.getMessage()); + } + } + return hiveAuthzBinding; + } + + private HiveConf getConf() { + return SessionState.get().getConf(); + } + + private void close() { + if (hiveAuthzBinding != null) { + hiveAuthzBinding.close(); + hiveAuthzBinding = null; + } + } +} http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java new file mode 100644 index 0000000..d1197a4 --- /dev/null +++ b/sentry-binding/sentry-binding-hive-common/src/main/java/org/apache/sentry/binding/metastore/SentryMetastorePostEventListenerBase.java @@ -0,0 +1,404 @@ +/** + * 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.sentry.binding.metastore; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreEventListener; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.events.AddPartitionEvent; +import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent; +import org.apache.hadoop.hive.metastore.events.AlterTableEvent; +import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent; +import org.apache.hadoop.hive.metastore.events.CreateTableEvent; +import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent; +import org.apache.hadoop.hive.metastore.events.DropPartitionEvent; +import org.apache.hadoop.hive.metastore.events.DropTableEvent; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.sentry.SentryUserException; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf; +import org.apache.sentry.binding.hive.conf.HiveAuthzConf.AuthzConfVars; +import org.apache.sentry.core.common.Authorizable; +import org.apache.sentry.core.model.db.Database; +import org.apache.sentry.core.model.db.Server; +import org.apache.sentry.core.model.db.Table; +import org.apache.sentry.provider.db.SentryMetastoreListenerPlugin; +import org.apache.sentry.provider.db.service.thrift.SentryPolicyServiceClient; +import org.apache.sentry.service.thrift.SentryServiceClientFactory; +import org.apache.sentry.service.thrift.ServiceConstants.ConfUtilties; +import org.apache.sentry.service.thrift.ServiceConstants.ServerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SentryMetastorePostEventListenerBase extends MetaStoreEventListener { + + private static final Logger LOGGER = LoggerFactory.getLogger(SentryMetastoreListenerPlugin.class); + private final HiveAuthzConf authzConf; + private final Server server; + + protected List<SentryMetastoreListenerPlugin> sentryPlugins = new ArrayList<SentryMetastoreListenerPlugin>(); + + public SentryMetastorePostEventListenerBase(Configuration config) { + super(config); + + if (!(config instanceof HiveConf)) { + String error = "Could not initialize Plugin - Configuration is not an instanceof HiveConf"; + LOGGER.error(error); + throw new RuntimeException(error); + } + + authzConf = HiveAuthzConf.getAuthzConf((HiveConf)config); + server = new Server(authzConf.get(AuthzConfVars.AUTHZ_SERVER_NAME.getVar())); + Iterable<String> pluginClasses = ConfUtilties.CLASS_SPLITTER + .split(config.get(ServerConfig.SENTRY_METASTORE_PLUGINS, + ServerConfig.SENTRY_METASTORE_PLUGINS_DEFAULT).trim()); + + try { + for (String pluginClassStr : pluginClasses) { + Class<?> clazz = config.getClassByName(pluginClassStr); + if (!SentryMetastoreListenerPlugin.class.isAssignableFrom(clazz)) { + throw new IllegalArgumentException("Class [" + + pluginClassStr + "] is not a " + + SentryMetastoreListenerPlugin.class.getName()); + } + SentryMetastoreListenerPlugin plugin = (SentryMetastoreListenerPlugin) clazz + .getConstructor(Configuration.class, Configuration.class) + .newInstance(config, authzConf); + sentryPlugins.add(plugin); + } + } catch (Exception e) { + LOGGER.error("Could not initialize Plugin !!", e); + throw new RuntimeException(e); + } + } + + @Override + public void onCreateTable (CreateTableEvent tableEvent) throws MetaException { + + // don't sync paths/privileges if the operation has failed + if (!tableEvent.getStatus()) { + LOGGER.debug("Skip sync paths/privileges with Sentry server for onCreateTable event," + + " since the operation failed. \n"); + return; + } + + if (tableEvent.getTable().getSd().getLocation() != null) { + String authzObj = tableEvent.getTable().getDbName() + "." + + tableEvent.getTable().getTableName(); + String path = tableEvent.getTable().getSd().getLocation(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.addPath(authzObj, path); + } + } + + // drop the privileges on the given table, in case if anything was left + // behind during the drop + if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) { + return; + } + + dropSentryTablePrivilege(tableEvent.getTable().getDbName(), + tableEvent.getTable().getTableName()); + } + + @Override + public void onDropTable(DropTableEvent tableEvent) throws MetaException { + + // don't sync paths/privileges if the operation has failed + if (!tableEvent.getStatus()) { + LOGGER.debug("Skip syncing paths/privileges with Sentry server for onDropTable event," + + " since the operation failed. \n"); + return; + } + + if (tableEvent.getTable().getSd().getLocation() != null) { + String authzObj = tableEvent.getTable().getDbName() + "." + + tableEvent.getTable().getTableName(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.removeAllPaths(authzObj, null); + } + } + // drop the privileges on the given table + if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) { + return; + } + + if (!tableEvent.getStatus()) { + return; + } + + dropSentryTablePrivilege(tableEvent.getTable().getDbName(), + tableEvent.getTable().getTableName()); + } + + @Override + public void onCreateDatabase(CreateDatabaseEvent dbEvent) + throws MetaException { + + // don't sync paths/privileges if the operation has failed + if (!dbEvent.getStatus()) { + LOGGER.debug("Skip syncing paths/privileges with Sentry server for onCreateDatabase event," + + " since the operation failed. \n"); + return; + } + + if (dbEvent.getDatabase().getLocationUri() != null) { + String authzObj = dbEvent.getDatabase().getName(); + String path = dbEvent.getDatabase().getLocationUri(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.addPath(authzObj, path); + } + } + // drop the privileges on the database, in case anything left behind during + // last drop db + if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_CREATE_WITH_POLICY_STORE)) { + return; + } + + dropSentryDbPrivileges(dbEvent.getDatabase().getName()); + } + + /** + * Drop the privileges on the database. Note that child tables will be + * dropped individually by client, so we just need to handle the removing + * the db privileges. The table drop should cleanup the table privileges. + */ + @Override + public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException { + + // don't sync paths/privileges if the operation has failed + if (!dbEvent.getStatus()) { + LOGGER.debug("Skip syncing paths/privileges with Sentry server for onDropDatabase event," + + " since the operation failed. \n"); + return; + } + + String authzObj = dbEvent.getDatabase().getName(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + List<String> tNames = dbEvent.getHandler().get_all_tables(authzObj); + plugin.removeAllPaths(authzObj, tNames); + } + if (!syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_DROP_WITH_POLICY_STORE)) { + return; + } + + dropSentryDbPrivileges(dbEvent.getDatabase().getName()); + } + + /** + * Adjust the privileges when table is renamed + */ + @Override + public void onAlterTable (AlterTableEvent tableEvent) throws MetaException { + + // don't sync privileges if the operation has failed + if (!tableEvent.getStatus()) { + LOGGER.debug("Skip syncing privileges with Sentry server for onAlterTable event," + + " since the operation failed. \n"); + return; + } + + renameSentryTablePrivilege(tableEvent.getOldTable().getDbName(), + tableEvent.getOldTable().getTableName(), + tableEvent.getOldTable().getSd().getLocation(), + tableEvent.getNewTable().getDbName(), + tableEvent.getNewTable().getTableName(), + tableEvent.getNewTable().getSd().getLocation()); + } + + @Override + public void onAlterPartition(AlterPartitionEvent partitionEvent) + throws MetaException { + + // don't sync privileges if the operation has failed + if (!partitionEvent.getStatus()) { + LOGGER.debug("Skip syncing privileges with Sentry server for onAlterPartition event," + + " since the operation failed. \n"); + return; + } + + String oldLoc = null, newLoc = null; + if (partitionEvent.getOldPartition() != null) { + oldLoc = partitionEvent.getOldPartition().getSd().getLocation(); + } + if (partitionEvent.getNewPartition() != null) { + newLoc = partitionEvent.getNewPartition().getSd().getLocation(); + } + + if (oldLoc != null && newLoc != null && !oldLoc.equals(newLoc)) { + String authzObj = + partitionEvent.getOldPartition().getDbName() + "." + + partitionEvent.getOldPartition().getTableName(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.renameAuthzObject(authzObj, oldLoc, + authzObj, newLoc); + } + } + } + + @Override + public void onAddPartition(AddPartitionEvent partitionEvent) + throws MetaException { + + // don't sync path if the operation has failed + if (!partitionEvent.getStatus()) { + LOGGER.debug("Skip syncing path with Sentry server for onAddPartition event," + + " since the operation failed. \n"); + return; + } + + for (Partition part : partitionEvent.getPartitions()) { + if (part.getSd() != null && part.getSd().getLocation() != null) { + String authzObj = part.getDbName() + "." + part.getTableName(); + String path = part.getSd().getLocation(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.addPath(authzObj, path); + } + } + } + super.onAddPartition(partitionEvent); + } + + @Override + public void onDropPartition(DropPartitionEvent partitionEvent) + throws MetaException { + + // don't sync path if the operation has failed + if (!partitionEvent.getStatus()) { + LOGGER.debug("Skip syncing path with Sentry server for onDropPartition event," + + " since the operation failed. \n"); + return; + } + + String authzObj = partitionEvent.getTable().getDbName() + "." + + partitionEvent.getTable().getTableName(); + String path = partitionEvent.getPartition().getSd().getLocation(); + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.removePath(authzObj, path); + } + super.onDropPartition(partitionEvent); + } + + private SentryPolicyServiceClient getSentryServiceClient() + throws MetaException { + try { + return SentryServiceClientFactory.create(authzConf); + } catch (Exception e) { + throw new MetaException("Failed to connect to Sentry service " + + e.getMessage()); + } + } + + private void dropSentryDbPrivileges(String dbName) throws MetaException { + List<Authorizable> authorizableTable = new ArrayList<Authorizable>(); + authorizableTable.add(server); + authorizableTable.add(new Database(dbName)); + try { + dropSentryPrivileges(authorizableTable); + } catch (SentryUserException e) { + throw new MetaException("Failed to remove Sentry policies for drop DB " + + dbName + " Error: " + e.getMessage()); + } catch (IOException e) { + throw new MetaException("Failed to find local user " + e.getMessage()); + } + + } + + private void dropSentryTablePrivilege(String dbName, String tabName) + throws MetaException { + List<Authorizable> authorizableTable = new ArrayList<Authorizable>(); + authorizableTable.add(server); + authorizableTable.add(new Database(dbName)); + authorizableTable.add(new Table(tabName)); + + try { + dropSentryPrivileges(authorizableTable); + } catch (SentryUserException e) { + throw new MetaException( + "Failed to remove Sentry policies for drop table " + dbName + "." + + tabName + " Error: " + e.getMessage()); + } catch (IOException e) { + throw new MetaException("Failed to find local user " + e.getMessage()); + } + + } + private void dropSentryPrivileges( + List<? extends Authorizable> authorizableTable) + throws SentryUserException, IOException, MetaException { + String requestorUserName = UserGroupInformation.getCurrentUser() + .getShortUserName(); + SentryPolicyServiceClient sentryClient = getSentryServiceClient(); + sentryClient.dropPrivileges(requestorUserName, authorizableTable); + + // Close the connection after dropping privileges is done. + sentryClient.close(); + } + + private void renameSentryTablePrivilege(String oldDbName, String oldTabName, + String oldPath, String newDbName, String newTabName, String newPath) + throws MetaException { + List<Authorizable> oldAuthorizableTable = new ArrayList<Authorizable>(); + oldAuthorizableTable.add(server); + oldAuthorizableTable.add(new Database(oldDbName)); + oldAuthorizableTable.add(new Table(oldTabName)); + + List<Authorizable> newAuthorizableTable = new ArrayList<Authorizable>(); + newAuthorizableTable.add(server); + newAuthorizableTable.add(new Database(newDbName)); + newAuthorizableTable.add(new Table(newTabName)); + + if (!oldTabName.equalsIgnoreCase(newTabName) + && syncWithPolicyStore(AuthzConfVars.AUTHZ_SYNC_ALTER_WITH_POLICY_STORE)) { + + SentryPolicyServiceClient sentryClient = getSentryServiceClient(); + + try { + String requestorUserName = UserGroupInformation.getCurrentUser() + .getShortUserName(); + sentryClient.renamePrivileges(requestorUserName, oldAuthorizableTable, newAuthorizableTable); + } catch (SentryUserException e) { + throw new MetaException( + "Failed to remove Sentry policies for rename table " + oldDbName + + "." + oldTabName + "to " + newDbName + "." + newTabName + + " Error: " + e.getMessage()); + } catch (IOException e) { + throw new MetaException("Failed to find local user " + e.getMessage()); + } finally { + + // Close the connection after renaming privileges is done. + sentryClient.close(); + } + } + // The HDFS plugin needs to know if it's a path change (set location) + for (SentryMetastoreListenerPlugin plugin : sentryPlugins) { + plugin.renameAuthzObject(oldDbName + "." + oldTabName, oldPath, + newDbName + "." + newTabName, newPath); + } + } + + private boolean syncWithPolicyStore(AuthzConfVars syncConfVar) { + return "true" + .equalsIgnoreCase(authzConf.get(syncConfVar.getVar(), "true")); + } + +} http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive-v2/pom.xml ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive-v2/pom.xml b/sentry-binding/sentry-binding-hive-v2/pom.xml index ef6048c..f33219d 100644 --- a/sentry-binding/sentry-binding-hive-v2/pom.xml +++ b/sentry-binding/sentry-binding-hive-v2/pom.xml @@ -31,7 +31,7 @@ limitations under the License. <dependencies> <dependency> <groupId>org.apache.sentry</groupId> - <artifactId>sentry-binding-hive</artifactId> + <artifactId>sentry-binding-hive-common</artifactId> <exclusions> <exclusion> <groupId>org.apache.httpcomponents</groupId> @@ -114,21 +114,6 @@ limitations under the License. <groupId>org.apache.sentry</groupId> <artifactId>sentry-provider-common</artifactId> </dependency> - <!-- required for SentryGrantRevokeTask --> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-provider-db</artifactId> - <exclusions> - <exclusion> - <groupId>org.apache.hive</groupId> - <artifactId>hive-beeline</artifactId> - </exclusion> - <exclusion> - <groupId>org.apache.hive</groupId> - <artifactId>hive-metastore</artifactId> - </exclusion> - </exclusions> - </dependency> <dependency> <groupId>org.apache.sentry</groupId> <artifactId>sentry-provider-file</artifactId> http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/pom.xml ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive/pom.xml b/sentry-binding/sentry-binding-hive/pom.xml index fb5f214..ff5882e 100644 --- a/sentry-binding/sentry-binding-hive/pom.xml +++ b/sentry-binding/sentry-binding-hive/pom.xml @@ -30,14 +30,6 @@ limitations under the License. <dependencies> <dependency> - <groupId>org.apache.thrift</groupId> - <artifactId>libthrift</artifactId> - </dependency> - <dependency> - <groupId>org.apache.derby</groupId> - <artifactId>derby</artifactId> - </dependency> - <dependency> <groupId>junit</groupId> <artifactId>junit</artifactId> <scope>test</scope> @@ -54,32 +46,7 @@ limitations under the License. </dependency> <dependency> <groupId>org.apache.sentry</groupId> - <artifactId>sentry-core-common</artifactId> - </dependency> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-core-model-db</artifactId> - </dependency> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-provider-common</artifactId> - </dependency> - <!-- required for SentryGrantRevokeTask --> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-provider-db</artifactId> - </dependency> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-provider-file</artifactId> - </dependency> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-provider-cache</artifactId> - </dependency> - <dependency> - <groupId>org.apache.sentry</groupId> - <artifactId>sentry-policy-db</artifactId> + <artifactId>sentry-binding-hive-common</artifactId> </dependency> <dependency> <groupId>org.apache.hadoop</groupId> @@ -97,6 +64,11 @@ limitations under the License. <artifactId>mockito-all</artifactId> <scope>test</scope> </dependency> + <!-- required for SentryGrantRevokeTask --> + <dependency> + <groupId>org.apache.sentry</groupId> + <artifactId>sentry-provider-db</artifactId> + </dependency> </dependencies> </project> http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java deleted file mode 100644 index 5238414..0000000 --- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/SentryHiveConstants.java +++ /dev/null @@ -1,31 +0,0 @@ -/** - * 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.hive; - -import java.util.EnumSet; - -import org.apache.hadoop.hive.ql.security.authorization.PrivilegeType; - -public class SentryHiveConstants { - public static final EnumSet<PrivilegeType> ALLOWED_PRIVS = EnumSet.allOf(PrivilegeType.class); - - public static final String PRIVILEGE_NOT_SUPPORTED = "Sentry does not support privilege: "; - public static final String PARTITION_PRIVS_NOT_SUPPORTED = "Sentry does not support partition level authorization"; - public static final String GRANT_REVOKE_NOT_SUPPORTED_ON_OBJECT = "Sentry does not allow grant/revoke on: "; - public static final String GRANT_REVOKE_NOT_SUPPORTED_FOR_PRINCIPAL = "Sentry does not allow privileges to be granted/revoked to/from: "; -} http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java deleted file mode 100644 index 8838368..0000000 --- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryFilterDDLTask.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.hive.ql.exec; - -import static org.apache.hadoop.util.StringUtils.stringifyException; - -import java.io.DataOutputStream; -import java.io.IOException; -import java.util.List; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.ql.DriverContext; -import org.apache.hadoop.hive.ql.ErrorMsg; -import org.apache.hadoop.hive.ql.metadata.Hive; -import org.apache.hadoop.hive.ql.metadata.HiveException; -import org.apache.hadoop.hive.ql.metadata.Table; -import org.apache.hadoop.hive.ql.metadata.formatting.MetaDataFormatUtils; -import org.apache.hadoop.hive.ql.plan.HiveOperation; -import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc; -import org.apache.hadoop.hive.ql.session.SessionState; -import org.apache.hadoop.io.IOUtils; -import org.apache.sentry.binding.hive.HiveAuthzBindingHook; -import org.apache.sentry.binding.hive.authz.HiveAuthzBinding; -import org.apache.sentry.core.common.Subject; - -import com.google.common.base.Preconditions; - -public class SentryFilterDDLTask extends DDLTask { - private static final long serialVersionUID = 1L; - private static final Log LOG = LogFactory.getLog(SentryFilterDDLTask.class); - - private HiveAuthzBinding hiveAuthzBinding; - private Subject subject; - private HiveOperation stmtOperation; - - public SentryFilterDDLTask(HiveAuthzBinding hiveAuthzBinding, Subject subject, - HiveOperation stmtOperation) { - Preconditions.checkNotNull(hiveAuthzBinding); - Preconditions.checkNotNull(subject); - Preconditions.checkNotNull(stmtOperation); - - this.hiveAuthzBinding = hiveAuthzBinding; - this.subject = subject; - this.stmtOperation = stmtOperation; - } - - public HiveAuthzBinding getHiveAuthzBinding() { - return hiveAuthzBinding; - } - - public Subject getSubject() { - return subject; - } - - public HiveOperation getStmtOperation() { - return stmtOperation; - } - - @Override - public int execute(DriverContext driverContext) { - // Currently the SentryFilterDDLTask only supports filter the "show columns in table " command. - ShowColumnsDesc showCols = work.getShowColumnsDesc(); - try { - if (showCols != null) { - return showFilterColumns(showCols); - } - } catch (Throwable e) { - failed(e); - return 1; - } - - return super.execute(driverContext); - } - - private void failed(Throwable e) { - while (e.getCause() != null && e.getClass() == RuntimeException.class) { - e = e.getCause(); - } - setException(e); - LOG.error(stringifyException(e)); - } - - /** - * Filter the command "show columns in table" - * - */ - private int showFilterColumns(ShowColumnsDesc showCols) throws HiveException { - Table table = Hive.get(conf).getTable(showCols.getTableName()); - - // write the results in the file - DataOutputStream outStream = null; - try { - Path resFile = new Path(showCols.getResFile()); - FileSystem fs = resFile.getFileSystem(conf); - outStream = fs.create(resFile); - - List<FieldSchema> cols = table.getCols(); - cols.addAll(table.getPartCols()); - // In case the query is served by HiveServer2, don't pad it with spaces, - // as HiveServer2 output is consumed by JDBC/ODBC clients. - boolean isOutputPadded = !SessionState.get().isHiveServerQuery(); - outStream.writeBytes(MetaDataFormatUtils.getAllColumnsInformation( - fiterColumns(cols, table), false, isOutputPadded, null)); - outStream.close(); - outStream = null; - } catch (IOException e) { - throw new HiveException(e, ErrorMsg.GENERIC_ERROR); - } finally { - IOUtils.closeStream(outStream); - } - return 0; - } - - private List<FieldSchema> fiterColumns(List<FieldSchema> cols, Table table) throws HiveException { - // filter some columns that the subject has privilege on - return HiveAuthzBindingHook.filterShowColumns(getHiveAuthzBinding(), - cols, getStmtOperation(), getSubject().getName(), table.getTableName(), table.getDbName()); - } -} http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java index 31eb5e8..cd8352a 100644 --- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java +++ b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryGrantRevokeTask.java @@ -55,7 +55,7 @@ import org.apache.hadoop.hive.ql.security.authorization.PrivilegeType; import org.apache.hadoop.hive.ql.session.SessionState; import org.apache.hadoop.hive.ql.session.SessionState.LogHelper; import org.apache.sentry.SentryUserException; -import org.apache.sentry.binding.hive.HiveAuthzBindingHook; +import org.apache.sentry.binding.hive.HiveAuthzBindingHookBase; import org.apache.sentry.binding.hive.SentryOnFailureHookContext; import org.apache.sentry.binding.hive.SentryOnFailureHookContextImpl; import org.apache.sentry.binding.hive.authz.HiveAuthzBinding; @@ -160,7 +160,7 @@ public class SentryGrantRevokeTask extends Task<DDLWork> implements Serializable new HashSet<WriteEntity>(), stmtOperation, null, null, null, null, subject.getName(), ipAddress, new AuthorizationException(e), conf); - HiveAuthzBindingHook.runFailureHook(hookContext, csHooks); + HiveAuthzBindingHookBase.runFailureHook(hookContext, csHooks); throw e; // rethrow the exception for logging } } catch(SentryUserException e) { http://git-wip-us.apache.org/repos/asf/sentry/blob/7a30c819/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java ---------------------------------------------------------------------- diff --git a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java b/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java deleted file mode 100644 index 4fa4221..0000000 --- a/sentry-binding/sentry-binding-hive/src/main/java/org/apache/hadoop/hive/ql/exec/SentryHivePrivilegeObjectDesc.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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.hive.ql.exec; - -import org.apache.hadoop.hive.ql.plan.PrivilegeObjectDesc; - -public class SentryHivePrivilegeObjectDesc extends PrivilegeObjectDesc { - private boolean isUri; - private boolean isServer; - - public SentryHivePrivilegeObjectDesc() { - // reset table type which is on by default - super.setTable(false); - } - - public boolean getUri() { - return isUri; - } - - public void setUri(boolean isUri) { - this.isUri = isUri; - } - - public boolean getServer() { - return isServer; - } - - public void setServer(boolean isServer) { - this.isServer = isServer; - } - - public boolean isSentryPrivObjectDesc() { - return isServer || isUri; - } - -}
