You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by bo...@apache.org on 2019/08/29 12:44:14 UTC

[impala] 07/07: IMPALA-8851: Do not throw authorization exception in drop if exists queries

This is an automated email from the ASF dual-hosted git repository.

boroknagyz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 026c5345e0e27383244d757660856e8a157aac20
Author: Vihang Karajgaonkar <vi...@cloudera.com>
AuthorDate: Tue Aug 13 11:54:42 2019 -0700

    IMPALA-8851: Do not throw authorization exception in drop if exists queries
    
    Note that this is the continuation of work in
    https://github.com/vihangk1/impala/commits/IMPALA-8851
    
    This patch's goal is to change Impala's behavior in the following case:
    - the query is a DROP TABLE/VIEW/DATABASE/FUNCTIONS IF EXISTS statement
    - the given object does not exist
    - the user has some kind of privilege on the object, which imples the
      privilege to know whether object exists, but does not have DROP
      privilege on the object
    
    Until now this lead to an authorization exception, while it will be
    allowed with this change.
    
    An example where this is useful is a user who has CREATE privilege on
    a database, and creates table t_owned, and gets ownership of the
    table. In this case DROP TABLE IF EXISTS was non idempotent:
    DROP TABLE IF EXISTS t_owned;
    -> success
    DROP TABLE IF EXISTS t_owned;
    -> authorization error, as the privileges for the table were
       deleted when the table was successfully dropped
    
    After this change the second statement will be also successful.
    
    The authorization logic has to avoid leaking information that the
    user has no right to know. For this reason DROP IF EXISTS has to
    return the same error message regardless whether the object exists
    or not if the user has no right to know it's existence. This is
    achieved with the following pattern:
    - in the IF EXISTS case first an ANY privilege is registered, then
      the existence of the object is checked and if it doesn't exist,
      the analysis returns successfully
    - if the object exists, the DROP privilege is registered (if there is
      no IF EXISTS in the query, this always happens)
    - as the authorization logic checks privileges in the order of
      registration, first the ANY will be checked, and DROP will be only
      checked if the user has ANY privileges
    
    Testing:
    - Added a new test case in the sentry tests which confirms that the
    authorization exception is not thrown when a drop if exists query is
    issued on a object which does not exist.
    - Changed several tests affected by the new behavior.
    - Ran core tests.
    
    Change-Id: Iba068935e5da92d71e16e2321afdb8e7b781086a
    Reviewed-on: http://gerrit.cloudera.org:8080/14121
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../java/org/apache/impala/analysis/Analyzer.java  |  17 +++
 .../org/apache/impala/analysis/DropDbStmt.java     |  17 ++-
 .../apache/impala/analysis/DropFunctionStmt.java   |  31 +++--
 .../impala/analysis/DropTableOrViewStmt.java       |  17 ++-
 .../apache/impala/service/CatalogOpExecutor.java   |  19 ++-
 .../authorization/AuthorizationStmtTest.java       |  61 ++++++---
 .../authorization/AuthorizationTestBase.java       |   4 +
 .../impala/testutil/TestSentryGroupMapper.java     |   1 +
 .../queries/QueryTest/create-database.test         |   2 +-
 tests/authorization/test_owner_privileges.py       | 143 +++++++++++++++++++++
 tests/common/sentry_cache_test_suite.py            |   6 +
 11 files changed, 280 insertions(+), 38 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
index 3936722..4f23fa0 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Analyzer.java
@@ -2651,6 +2651,23 @@ public class Analyzer {
   }
 
   /**
+   * Checks if a table exists without registering privileges.
+   */
+  public boolean tableExists(TableName tblName) {
+    Preconditions.checkNotNull(tblName);
+    TableName fqTableName = getFqTableName(tblName);
+    return globalState_.stmtTableCache.tables.containsKey(fqTableName);
+  }
+
+  /**
+   * Checks if a database exists without registering privileges.
+   */
+  public boolean dbExists(String dbName) {
+    Preconditions.checkNotNull(dbName);
+    return getCatalog().getDb(dbName) != null;
+  }
+
+  /**
    * Returns the Table with the given name from the 'loadedTables' map in the global
    * analysis state. Throws an AnalysisException if the table or the db does not exist.
    * Throws a TableLoadingException if the registered table failed to load.
diff --git a/fe/src/main/java/org/apache/impala/analysis/DropDbStmt.java b/fe/src/main/java/org/apache/impala/analysis/DropDbStmt.java
index 68aa5ba..386579f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DropDbStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DropDbStmt.java
@@ -68,7 +68,19 @@ public class DropDbStmt extends StatementBase {
 
   @Override
   public void analyze(Analyzer analyzer) throws AnalysisException {
-    FeDb db = analyzer.getDb(dbName_, Privilege.DROP, false);
+    // Set the servername here if authorization is enabled because analyzer_ is not
+    // available in the toThrift() method.
+    serverName_ = analyzer.getServerName();
+    if (ifExists_) {
+      // Start with ANY privilege in case of IF EXISTS, and register DROP privilege
+      // later only if the database exists. See IMPALA-8851 for more explanation.
+      analyzer.registerPrivReq(builder ->
+          builder.allOf(Privilege.ANY)
+              .onDb(dbName_)
+              .build());
+      if (!analyzer.dbExists(dbName_)) return;
+    }
+    FeDb db = analyzer.getDb(dbName_, Privilege.DROP, false, false);
     if (db == null && !ifExists_) {
       throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + dbName_);
     }
@@ -79,8 +91,5 @@ public class DropDbStmt extends StatementBase {
     if (db != null && db.numFunctions() > 0 && !cascade_) {
       throw new AnalysisException("Cannot drop non-empty database: " + dbName_);
     }
-    // Set the servername here if authorization is enabled because analyzer_ is not
-    // available in the toThrift() method.
-    serverName_ = analyzer.getServerName();
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/DropFunctionStmt.java b/fe/src/main/java/org/apache/impala/analysis/DropFunctionStmt.java
index b05c286..b87b9f6 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DropFunctionStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DropFunctionStmt.java
@@ -83,26 +83,37 @@ public class DropFunctionStmt extends StatementBase {
           false);
     }
 
-    analyzer.registerPrivReq(builder ->
-        builder.onFunction(desc_.dbName(), desc_.signatureString())
-            .allOf(Privilege.DROP)
-            .build());
+    // Start with ANY privilege in case of IF EXISTS, and register DROP privilege later
+    // only if the function exists. See IMPALA-8851 for more explanation.
+    registerFnPriv(analyzer, ifExists_ ? Privilege.ANY : Privilege.DROP);
 
     FeDb db =  analyzer.getDb(desc_.dbName(), false);
-    if (db == null && !ifExists_) {
+    if (db == null) {
+      if (ifExists_) return;
+      // db does not exist and if exists clause is not provided
       throw new AnalysisException(Analyzer.DB_DOES_NOT_EXIST_ERROR_MSG + desc_.dbName());
     }
-
-    if (!hasSignature() && db != null && db.getFunctions(
-        desc_.functionName()).isEmpty() && !ifExists_) {
+    if (!hasSignature() && db.getFunctions(desc_.functionName()).isEmpty()) {
+      if (ifExists_) return;
       throw new AnalysisException(
           Analyzer.FN_DOES_NOT_EXIST_ERROR_MSG + desc_.functionName());
     }
-
     if (hasSignature() && analyzer.getCatalog().getFunction(
-        desc_, Function.CompareMode.IS_IDENTICAL) == null && !ifExists_) {
+        desc_, Function.CompareMode.IS_IDENTICAL) == null) {
+      if (ifExists_) return;
       throw new AnalysisException(
           Analyzer.FN_DOES_NOT_EXIST_ERROR_MSG + desc_.signatureString());
     }
+
+    // Register the "stronger" DROP privilege if only ANY was registered due to
+    // IF EXISTS.
+    if (ifExists_) registerFnPriv(analyzer, Privilege.DROP);
+  }
+
+  private void registerFnPriv(Analyzer analyzer, Privilege priv) {
+    analyzer.registerPrivReq(builder ->
+          builder.onFunction(desc_.dbName(), desc_.signatureString())
+              .allOf(priv)
+              .build());
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
index 86da283..cf80569 100644
--- a/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/DropTableOrViewStmt.java
@@ -107,14 +107,27 @@ public class DropTableOrViewStmt extends StatementBase {
     // available in the toThrift() method.
     serverName_ = analyzer.getServerName();
     try {
+      if (ifExists_) {
+        // Start with ANY privilege in case of IF EXISTS, and register DROP privilege
+        // later only if the table exists. See IMPALA-8851 for more explanation.
+        analyzer.registerPrivReq(builder ->
+            builder.allOf(Privilege.ANY)
+                .onTable(dbName_, getTbl())
+                .build());
+        if (!analyzer.tableExists(tableName_)) return;
+      }
       FeTable table = analyzer.getTable(tableName_, /* add access event */ true,
           /* add column-level privilege */ false, Privilege.DROP);
       Preconditions.checkNotNull(table);
       if (table instanceof FeView && dropTable_) {
+        // DROP VIEW IF EXISTS 'table' succeeds, similarly to Hive, but unlike postgres.
+        if (ifExists_) return;
         throw new AnalysisException(String.format(
             "DROP TABLE not allowed on a view: %s.%s", dbName_, getTbl()));
       }
       if (!(table instanceof FeView) && !dropTable_) {
+        // DROP TABLE IF EXISTS 'view' succeeds, similarly to Hive, but unlike postgres.
+        if (ifExists_) return;
         throw new AnalysisException(String.format(
             "DROP VIEW not allowed on a table: %s.%s", dbName_, getTbl()));
       }
@@ -123,7 +136,6 @@ public class DropTableOrViewStmt extends StatementBase {
         analyzer.checkTableCapability(table, Analyzer.OperationType.WRITE);
         analyzer.ensureTableNotFullAcid(table);
       }
-
     } catch (TableLoadingException e) {
       // We should still try to DROP tables that failed to load, so that tables that are
       // in a bad state, eg. deleted externally from Kudu, can be dropped.
@@ -134,9 +146,6 @@ public class DropTableOrViewStmt extends StatementBase {
           analyzer.getFqTableName(tableName_).toString(), TCatalogObjectType.TABLE,
           Privilege.DROP.toString()));
       LOG.info("Ignoring TableLoadingException for {}", tableName_);
-    } catch (AnalysisException e) {
-      if (!ifExists_) throw e;
-      LOG.info("Ignoring AnalysisException for {}", tableName_);
     }
   }
 
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 64b1ae1..ed301b0 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -1509,12 +1509,22 @@ public class CatalogOpExecutor {
       // The Kudu tables in the HMS should have been dropped at this point
       // with the Hive Metastore integration enabled.
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
+        // HMS client does not have a way to identify if the database was dropped or
+        // not if the ignoreIfUnknown flag is true. Hence we always pass the
+        // ignoreIfUnknown as false and catch the NoSuchObjectFoundException and
+        // determine if we should throw or not
         msClient.getHiveClient().dropDatabase(
-            params.getDb(), true, params.if_exists, params.cascade);
+            params.getDb(), /* deleteData */true, /* ignoreIfUnknown */false,
+            params.cascade);
         addSummary(resp, "Database has been dropped.");
       } catch (TException e) {
-        throw new ImpalaRuntimeException(
-            String.format(HMS_RPC_ERROR_FORMAT_STR, "dropDatabase"), e);
+        if (e instanceof NoSuchObjectException && params.if_exists) {
+          // if_exists param was set; we ignore the NoSuchObjectFoundException
+          addSummary(resp, "Database does not exist.");
+        } else {
+          throw new ImpalaRuntimeException(
+              String.format(HMS_RPC_ERROR_FORMAT_STR, "dropDatabase"), e);
+        }
       }
       Db removedDb = catalog_.removeDb(params.getDb());
 
@@ -1538,6 +1548,9 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(removedObject);
     resp.result.setVersion(removedObject.getCatalog_version());
     resp.result.addToRemoved_catalog_objects(removedObject);
+    // it is possible that HMS database has been removed out of band externally. In
+    // such a case we still would want to add the summary of the operation as database
+    // has been dropped since we cleaned up state from CatalogServer
     addSummary(resp, "Database has been dropped.");
   }
 
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
index ec957e1..f839138 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -1904,9 +1904,12 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
       test.ok(onServer(TPrivilegeLevel.ALL))
           .ok(onServer(TPrivilegeLevel.OWNER))
           .ok(onServer(TPrivilegeLevel.DROP))
-          .error(dropError("nodb"))
-          .error(dropError("nodb"), onServer(allExcept(
-              TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)));
+          .ok(onServer(TPrivilegeLevel.CREATE))
+          .ok(onServer(TPrivilegeLevel.SELECT))
+          .ok(onServer(TPrivilegeLevel.INSERT))
+          .ok(onServer(TPrivilegeLevel.ALTER))
+          .ok(onServer(TPrivilegeLevel.REFRESH))
+          .error(accessError("nodb"));
     }
 
     // Dropping system database is not allowed even if with ALL/OWNER privilege on server.
@@ -1956,14 +1959,20 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
         .ok(onServer(TPrivilegeLevel.ALL))
         .ok(onServer(TPrivilegeLevel.OWNER))
         .ok(onServer(TPrivilegeLevel.DROP))
+        .ok(onServer(TPrivilegeLevel.CREATE))
+        .ok(onServer(TPrivilegeLevel.SELECT))
+        .ok(onServer(TPrivilegeLevel.INSERT))
+        .ok(onServer(TPrivilegeLevel.ALTER))
+        .ok(onServer(TPrivilegeLevel.REFRESH))
         .ok(onDatabase("functional", TPrivilegeLevel.ALL))
         .ok(onDatabase("functional", TPrivilegeLevel.OWNER))
         .ok(onDatabase("functional", TPrivilegeLevel.DROP))
-        .error(dropError("functional.notbl"))
-        .error(dropError("functional.notbl"), onServer(allExcept(
-            TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)))
-        .error(dropError("functional.notbl"), onDatabase("functional",
-            allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)));
+        .ok(onDatabase("functional", TPrivilegeLevel.CREATE))
+        .ok(onDatabase("functional", TPrivilegeLevel.SELECT))
+        .ok(onDatabase("functional", TPrivilegeLevel.INSERT))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+        .ok(onDatabase("functional", TPrivilegeLevel.REFRESH))
+        .error(accessError("functional.notbl"));
 
     // Dropping any tables in the system database is not allowed even with ALL/OWNER
     // privilege on server.
@@ -2014,14 +2023,20 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
         .ok(onServer(TPrivilegeLevel.ALL))
         .ok(onServer(TPrivilegeLevel.OWNER))
         .ok(onServer(TPrivilegeLevel.DROP))
+        .ok(onServer(TPrivilegeLevel.CREATE))
+        .ok(onServer(TPrivilegeLevel.SELECT))
+        .ok(onServer(TPrivilegeLevel.INSERT))
+        .ok(onServer(TPrivilegeLevel.ALTER))
+        .ok(onServer(TPrivilegeLevel.REFRESH))
         .ok(onDatabase("functional", TPrivilegeLevel.ALL))
         .ok(onDatabase("functional", TPrivilegeLevel.OWNER))
         .ok(onDatabase("functional", TPrivilegeLevel.DROP))
-        .error(dropError("functional.noview"))
-        .error(dropError("functional.noview"), onServer(allExcept(
-            TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)))
-        .error(dropError("functional.noview"), onDatabase("functional",
-            allExcept(TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)));
+        .ok(onDatabase("functional", TPrivilegeLevel.CREATE))
+        .ok(onDatabase("functional", TPrivilegeLevel.SELECT))
+        .ok(onDatabase("functional", TPrivilegeLevel.INSERT))
+        .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+        .ok(onDatabase("functional", TPrivilegeLevel.REFRESH))
+        .error(accessError("functional.noview"));
 
     // Dropping any views in the system database is not allowed even with ALL/OWNER
     // privilege on server.
@@ -2742,9 +2757,23 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
 
       // Function does not exist but with if exists clause.
       authorize("drop function if exists functional.g()")
-          .error(dropFunctionError("functional.g()"))
-          .error(dropFunctionError("functional.g()"), onServer(allExcept(
-              TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.DROP)));
+          .ok(onServer(TPrivilegeLevel.ALL))
+          .ok(onServer(TPrivilegeLevel.OWNER))
+          .ok(onServer(TPrivilegeLevel.DROP))
+          .ok(onServer(TPrivilegeLevel.CREATE))
+          .ok(onServer(TPrivilegeLevel.SELECT))
+          .ok(onServer(TPrivilegeLevel.INSERT))
+          .ok(onServer(TPrivilegeLevel.ALTER))
+          .ok(onServer(TPrivilegeLevel.REFRESH))
+          .ok(onDatabase("functional", TPrivilegeLevel.ALL))
+          .ok(onDatabase("functional", TPrivilegeLevel.OWNER))
+          .ok(onDatabase("functional", TPrivilegeLevel.DROP))
+          .ok(onDatabase("functional", TPrivilegeLevel.CREATE))
+          .ok(onDatabase("functional", TPrivilegeLevel.SELECT))
+          .ok(onDatabase("functional", TPrivilegeLevel.INSERT))
+          .ok(onDatabase("functional", TPrivilegeLevel.ALTER))
+          .ok(onDatabase("functional", TPrivilegeLevel.REFRESH))
+          .error(accessFunctionError("functional.g()"));
     } finally {
       removeFunction(fn);
     }
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
index a4787f3..6b968db 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
@@ -672,6 +672,10 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
     return "User '%s' does not have privileges to DROP functions in: " + object;
   }
 
+  protected static String accessFunctionError(String object) {
+    return "User '%s' does not have privileges to ANY functions in: " + object;
+  }
+
   protected static String columnMaskError(String object) {
     return "Impala does not support column masking yet. Column masking is enabled on " +
         "column: " + object;
diff --git a/fe/src/test/java/org/apache/impala/testutil/TestSentryGroupMapper.java b/fe/src/test/java/org/apache/impala/testutil/TestSentryGroupMapper.java
index 8ed8a7e..3237bde 100644
--- a/fe/src/test/java/org/apache/impala/testutil/TestSentryGroupMapper.java
+++ b/fe/src/test/java/org/apache/impala/testutil/TestSentryGroupMapper.java
@@ -58,6 +58,7 @@ public class TestSentryGroupMapper implements GroupMappingService {
     // User to groups for test_owner_privilege tests.
     groupsMap_.put("oo_user1", Sets.newHashSet("oo_group1"));
     groupsMap_.put("oo_user2", Sets.newHashSet("oo_group2"));
+    groupsMap_.put("oo_user3", Sets.newHashSet("oo_group3"));
 
     groupsMap_.put("foobar", Sets.newHashSet("foobar"));
     groupsMap_.put("FOOBAR", Sets.newHashSet("FOOBAR"));
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-database.test b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
index 433c50f..50765de 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-database.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
@@ -53,7 +53,7 @@ STRING, STRING
 # Dropping a non-existent databases is ok with IF EXISTS
 drop database if exists $DATABASE_2
 ---- RESULTS
-'Database has been dropped.'
+'Database does not exist.'
 ====
 ---- QUERY
 # Test DROP DATABASE ... CASCADE
diff --git a/tests/authorization/test_owner_privileges.py b/tests/authorization/test_owner_privileges.py
index 4a183e9..0c3fd44 100644
--- a/tests/authorization/test_owner_privileges.py
+++ b/tests/authorization/test_owner_privileges.py
@@ -127,6 +127,149 @@ class TestOwnerPrivileges(SentryCacheTestSuite):
                     "org.apache.impala.testutil.TestSentryResourceAuthorizationProvider"
                     .format(SENTRY_CONFIG_FILE_OO, SENTRY_LONG_POLLING_FREQUENCY_S),
       sentry_config=SENTRY_CONFIG_FILE_OO,
+      sentry_log_dir="{0}/test_drop_table_if_exists".format(SENTRY_BASE_LOG_DIR))
+  def test_drop_if_exists(self, vector, unique_database):
+    try:
+      # we need to use a separate db for testing the drop database if exists case
+      # For others we can rely on using unique_database which gets cleaned up
+      # automatically
+      test_db = "test_drop_if_exists_db"
+      self._setup_drop_if_exist_test(unique_database, test_db)
+      self._execute_drop_if_exists(TestObject(TestObject.DATABASE, test_db))
+      self._execute_drop_if_exists(TestObject(TestObject.TABLE, unique_database +
+          ".test_table"))
+      self._execute_drop_if_exists(TestObject(TestObject.VIEW, unique_database +
+          ".test_view"))
+      self._execute_drop_if_exists(TestObject(TestObject.FUNCTION, unique_database +
+          ".test_func()"))
+    finally:
+      self._cleanup_drop_if_exist_test(test_db)
+
+  def _setup_drop_if_exist_test(self, unique_database, test_db):
+    # Cleanup test_db, as the previous test run may have not been able to clean it up
+    self.execute_query("drop database if exists %s" % test_db)
+
+    # create a role which can create objects on this server
+    # for the purposes of this test oo_user1 will have privileges
+    # to create (and drop) objects while oo_user_2 will not have drop privileges
+    # oo_user_3 will have privileges to select on the server but not to drop
+    for role_name in self.execute_query("show roles").data:
+      if role_name in ['oo_user1_role', 'oo_user2_role', 'oo_user3_role']:
+        self.execute_query("drop role {0}".format(role_name))
+
+    self.execute_query("create role oo_user1_role")
+    self.execute_query("create role oo_user2_role")
+    self.execute_query("create role oo_user3_role")
+    # grant create permissions to oo_user_1 so that they can create database/table
+    # or functions
+    self.execute_query("grant create on server to oo_user1_role")
+    # grant select permissions to oo_user_3 so that they can use database/table
+    # or functions, but cannot drop them
+    self.execute_query("grant select on server to oo_user3_role")
+    # oo_user1 needs permissions to create a view based of functional.alltypes
+    self.execute_query("grant select on database functional to oo_user1_role")
+    # oo_user1 needs permissions to create a function based of libTestUdfs.so
+    self.execute_query("grant all on uri 'hdfs:///test-warehouse/libTestUdfs.so' to"
+        " oo_user1_role")
+    # We need to provide explicit privileges to drop functions
+    self.execute_query("grant drop on database {0} to "
+        "oo_user1_role".format(unique_database))
+    self.execute_query("grant role oo_user1_role to group oo_group1")
+    self.execute_query("grant role oo_user2_role to group oo_group2")
+    self.execute_query("grant role oo_user3_role to group oo_group3")
+    self.execute_query("refresh authorization")
+
+  def _cleanup_drop_if_exist_test(self, test_db):
+    self.execute_query("revoke role oo_user1_role from group oo_group1")
+    self.execute_query("revoke role oo_user2_role from group oo_group2")
+    self.execute_query("revoke role oo_user3_role from group oo_group3")
+    self.execute_query("drop role oo_user1_role")
+    self.execute_query("drop role oo_user2_role")
+    self.execute_query("drop role oo_user3_role")
+    self.execute_query("refresh authorization")
+    self.execute_query("drop database if exists {0}".format(test_db))
+
+  def _execute_drop_if_exists(self, test_obj):
+    self._execute_drop_if_exists_inner(test_obj, True)
+    if test_obj.obj_type != TestObject.DATABASE:
+      self._execute_drop_if_exists_inner(test_obj, False)
+
+  def _execute_drop_if_exists_inner(self, test_obj, use_qualified_name):
+    """
+    Executes a drop table if exists on a given object type and makes sure that
+    there is no authorization exception when the user has enough privileges. If the user
+    does not have correct privileges the test confirms that error is thrown
+    """
+    self.oo_user1_impalad_client = self.create_impala_client()
+    self.oo_user2_impalad_client = self.create_impala_client()
+    self.oo_user3_impalad_client = self.create_impala_client()
+
+    fq_obj_name = test_obj.obj_name
+    obj_name = fq_obj_name if use_qualified_name else test_obj.table_name
+    if not use_qualified_name:
+       # Call "use db" and omit "db." in the queries of oo_user1 and oo_user3
+       # oo_user2 has no privileges for the database so it will always use qualified name
+       self.user_query(self.oo_user1_impalad_client, "use %s" % test_obj.db_name,
+                       user="oo_user1")
+       self.user_query(self.oo_user3_impalad_client, "use %s" % test_obj.db_name,
+                       user="oo_user3")
+
+    self.user_query(self.oo_user1_impalad_client, "create %s %s %s %s %s" %
+                    (test_obj.obj_type, test_obj.obj_name, test_obj.table_def,
+                     test_obj.view_select, test_obj.func_def), user="oo_user1")
+
+    access_error_msg = \
+        "does not have privileges to ANY" if test_obj.obj_type == TestObject.FUNCTION \
+        else "does not have privileges to access"
+    drop_error_msg = \
+        "does not have privileges to DROP" if test_obj.obj_type == TestObject.FUNCTION \
+        else "does not have privileges to execute 'DROP'"
+
+    # Try to DROP IF EXISTS an existing object without DROP privileges
+    self.user_query(self.oo_user2_impalad_client, "drop %s if exists %s" %
+                    (test_obj.obj_type, fq_obj_name), user="oo_user2,",
+                    error_msg=access_error_msg)
+    self.user_query(self.oo_user3_impalad_client, "drop %s if exists %s" %
+                    (test_obj.obj_type, obj_name), user="oo_user3",
+                    error_msg=drop_error_msg)
+
+    # Try to DROP (without IF EXISTS) an existing object
+    self.user_query(self.oo_user2_impalad_client, "drop %s %s" %
+                    (test_obj.obj_type, fq_obj_name), user="oo_user2",
+                    error_msg=drop_error_msg)
+    self.user_query(self.oo_user3_impalad_client, "drop %s %s" %
+                    (test_obj.obj_type, obj_name), user="oo_user3",
+                    error_msg=drop_error_msg)
+    # oo_user1 has the privileges to drop the object, so the next query drops it
+    self.user_query(self.oo_user1_impalad_client, "drop %s %s" %
+                    (test_obj.obj_type, obj_name), user="oo_user1")
+
+    # a drop if exists on a non-existing object should not error out if the user has
+    # minimum set of privileges required to list those object types
+    self.user_query(self.oo_user1_impalad_client, "drop %s if exists %s" %
+                    (test_obj.obj_type, obj_name), user="oo_user1")
+    self.user_query(self.oo_user3_impalad_client, "drop %s if exists %s" %
+                    (test_obj.obj_type, obj_name), user="oo_user3")
+    # oo_user2 does not have privileges on this object and hence should receive a
+    # authorization error.
+    error_msg = \
+        "does not have privileges to ANY" if test_obj.obj_type == TestObject.FUNCTION \
+        else "does not have privileges to access"
+    self.user_query(self.oo_user2_impalad_client, "drop %s if exists %s" %
+                    (test_obj.obj_type, fq_obj_name), user="oo_user2",
+                    error_msg=error_msg)
+
+  @pytest.mark.execute_serially
+  @SentryCacheTestSuite.with_args(
+      impalad_args="--server_name=server1 --sentry_config={0} "
+                   "--authorization_policy_provider_class="
+                   "org.apache.impala.testutil.TestSentryResourceAuthorizationProvider"
+                   .format(SENTRY_CONFIG_FILE_OO),
+      catalogd_args="--sentry_config={0} --sentry_catalog_polling_frequency_s={1} "
+                    "--authorization_policy_provider_class="
+                    "org.apache.impala.testutil.TestSentryResourceAuthorizationProvider"
+                    .format(SENTRY_CONFIG_FILE_OO, SENTRY_LONG_POLLING_FREQUENCY_S),
+      sentry_config=SENTRY_CONFIG_FILE_OO,
       sentry_log_dir="{0}/test_owner_privileges_with_grant".format(SENTRY_BASE_LOG_DIR))
   def test_owner_privileges_with_grant(self, vector, unique_database):
     """Tests owner privileges with grant on database, table, and view.
diff --git a/tests/common/sentry_cache_test_suite.py b/tests/common/sentry_cache_test_suite.py
index 2890911..2f0096a 100644
--- a/tests/common/sentry_cache_test_suite.py
+++ b/tests/common/sentry_cache_test_suite.py
@@ -99,6 +99,7 @@ class TestObject(object):
   DATABASE = "database"
   TABLE = "table"
   VIEW = "view"
+  FUNCTION = "function"
 
   def __init__(self, obj_type, obj_name="", grant=False):
     self.obj_name = obj_name
@@ -108,6 +109,7 @@ class TestObject(object):
     self.table_name = None
     self.table_def = ""
     self.view_select = ""
+    self.func_def = ""
     if len(parts) > 1:
       self.table_name = parts[1]
     if obj_type == TestObject.VIEW:
@@ -116,6 +118,10 @@ class TestObject(object):
     elif obj_type == TestObject.TABLE:
       self.grant_name = TestObject.TABLE
       self.table_def = "(col1 int)"
+    elif obj_type == TestObject.FUNCTION:
+      self.grant_name = TestObject.FUNCTION
+      self.func_def = "RETURNS INT LOCATION 'hdfs:///test-warehouse/libTestUdfs.so'\
+       SYMBOL='Fn'"
     else:
       self.grant_name = obj_type
     self.grant = grant