You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by st...@apache.org on 2023/12/11 07:13:57 UTC

(impala) 03/03: IMPALA-12398: Fix Ranger role not exists when altering db/table/view owner to a role

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

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

commit d9c067aa89313547c1d8dbf3840ebe308726f8c3
Author: jichen0919 <ji...@163.com>
AuthorDate: Mon Sep 25 23:02:36 2023 +0800

    IMPALA-12398: Fix Ranger role not exists when altering db/table/view owner to a role
    
    When Role '<ROLE_NAME>' is created with Ranger authorization enabled,
    if 'ALTER TABLE <TABLE_NAME> SET OWNER ROLE <ROLE_NAME>' statement is
    executed to assign role as the owner of the table, it will throw
    AnalysisException:Role '<ROLE_NAME>' does not exist.
    
    Before this patch, given the ALTER DATABASE/TABLE/VIEW SET OWNER ROLE
    statement, Impala always checked the existence of the given role in
    its AuthorizationPolicy. However, when the support for role-related
    statements with Ranger was added in IMPALA-10211, we only added the
    roles in RangerImpalaPlugin instead of AuthorizationPolicy.
    Therefore, the statement above  would fail even though an authorized
    user tries to set the owner to an existing role in RangerImpalaPlugin.
    
    This patch will directly use ranger impala plugin to check the
    existence of the role, instead of using AuthorizationPolicy object.
    
    Tests:
     - Pass unit tests. test method testAlterView in AuthorizationStmtTest
       is updated accordingly.
     - Pass e2e tests. test method _test_ownership in test_ranger.py is
       updated to cover the new implementation.
     - Pass core tests with ranger enabled.
    
    Change-Id: I2b029bdb90111dbd0eab5189360cc81090225cda
    Reviewed-on: http://gerrit.cloudera.org:8080/20508
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../impala/analysis/AlterDbSetOwnerStmt.java       |  12 ++-
 .../analysis/AlterTableOrViewSetOwnerStmt.java     |  12 ++-
 .../impala/authorization/AuthorizationChecker.java |   5 +
 .../authorization/NoopAuthorizationFactory.java    |   6 ++
 .../ranger/RangerAuthorizationChecker.java         |   5 +
 .../impala/authorization/ranger/RangerUtil.java    |   9 ++
 .../authorization/AuthorizationStmtTest.java       |  12 +++
 .../authorization/AuthorizationTestBase.java       |   7 +-
 .../CatalogServiceTestCatalogWithRanger.java       | 104 +++++++++++++++++++++
 .../org/apache/impala/common/FrontendTestBase.java |   5 +
 .../impala/testutil/CatalogServiceTestCatalog.java |  21 +++--
 tests/authorization/test_ranger.py                 |  13 +++
 12 files changed, 197 insertions(+), 14 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterDbSetOwnerStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterDbSetOwnerStmt.java
index 84cd0b7c1..01c960c19 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterDbSetOwnerStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterDbSetOwnerStmt.java
@@ -18,6 +18,8 @@
 package org.apache.impala.analysis;
 
 import com.google.common.base.Preconditions;
+import org.apache.impala.authorization.AuthorizationChecker;
+import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TAlterDbParams;
@@ -55,9 +57,13 @@ public class AlterDbSetOwnerStmt extends AlterDbStmt {
     // We don't allow assigning to a non-existent role because Ranger should know about
     // all roles. Ranger does not track all users so we allow assigning to a user
     // that Ranger doesn't know about yet.
-    if (analyzer.isAuthzEnabled() && owner_.getOwnerType() == TOwnerType.ROLE
-        && analyzer.getCatalog().getAuthPolicy().getRole(ownerName) == null) {
-      throw new AnalysisException(String.format("Role '%s' does not exist.", ownerName));
+    if (analyzer.isAuthzEnabled() && owner_.getOwnerType() == TOwnerType.ROLE) {
+      AuthorizationFactory authzFactory = analyzer.getAuthzFactory();
+      AuthorizationChecker authzChecker = authzFactory.newAuthorizationChecker();
+      if (!authzChecker.roleExists(ownerName)) {
+        throw new AnalysisException(
+            String.format("Role '%s' does not exist.", ownerName));
+      }
     }
     // Set the servername here if authorization is enabled because analyzer_ is not
     // available in the toThrift() method.
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
index baed48408..25d9ecb12 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterTableOrViewSetOwnerStmt.java
@@ -18,6 +18,8 @@
 package org.apache.impala.analysis;
 
 import com.google.common.base.Preconditions;
+import org.apache.impala.authorization.AuthorizationChecker;
+import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.thrift.TAlterTableOrViewSetOwnerParams;
@@ -52,9 +54,13 @@ public abstract class AlterTableOrViewSetOwnerStmt extends AlterTableStmt {
     // We don't allow assigning to a non-existent role because Ranger should know about
     // all roles. Ranger does not track all users so we allow assigning to a user
     // that Ranger doesn't know about yet.
-    if (analyzer.isAuthzEnabled() && owner_.getOwnerType() == TOwnerType.ROLE
-        && analyzer.getCatalog().getAuthPolicy().getRole(ownerName) == null) {
-      throw new AnalysisException(String.format("Role '%s' does not exist.", ownerName));
+    if (analyzer.isAuthzEnabled() && owner_.getOwnerType() == TOwnerType.ROLE) {
+      AuthorizationFactory authzFactory = analyzer.getAuthzFactory();
+      AuthorizationChecker authzChecker = authzFactory.newAuthorizationChecker();
+      if (!authzChecker.roleExists(ownerName)) {
+        throw new AnalysisException(
+            String.format("Role '%s' does not exist.", ownerName));
+      }
     }
 
     tableName_ = analyzer.getFqTableName(tableName_);
diff --git a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
index 10d698b2c..d64f1574f 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
@@ -112,4 +112,9 @@ public interface AuthorizationChecker {
    * This method is to be executed after AnalysisContext#analyze() is completed.
    */
   void postAnalyze(AuthorizationContext authzCtx);
+
+  /**
+   * This method returns whether the role exists for given role
+   */
+  boolean roleExists(String roleName);
 }
diff --git a/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java b/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java
index e3097f6a5..23733f680 100644
--- a/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java
+++ b/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java
@@ -239,6 +239,12 @@ public class NoopAuthorizationFactory implements AuthorizationFactory {
       @Override
       public void postAnalyze(AuthorizationContext authzCtx) {
       }
+
+      @Override
+      public boolean roleExists(String roleName) {
+        throw new UnsupportedOperationException(
+            String.format("%s is not supported", ClassUtil.getMethodName()));
+      }
     };
   }
 
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
index 046e3e99e..56f009f0c 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerAuthorizationChecker.java
@@ -754,4 +754,9 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
 
   @VisibleForTesting
   public RangerImpalaPlugin getRangerImpalaPlugin() { return plugin_; }
+
+  @Override
+  public boolean roleExists(String roleName) {
+    return RangerUtil.roleExists(plugin_, roleName);
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerUtil.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerUtil.java
index a7bf26d50..022236c11 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerUtil.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerUtil.java
@@ -19,13 +19,16 @@ package org.apache.impala.authorization.ranger;
 
 import com.google.common.collect.Sets;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.service.BackendConfig;
 import org.apache.impala.thrift.TPrivilege;
+import org.apache.ranger.plugin.model.RangerRole;
 
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Predicate;
 
 /**
  * Collection of static functions to support Apache Ranger implementation
@@ -119,4 +122,10 @@ public class RangerUtil {
       throws Exception {
     plugin.getAllRoles(user, null);
   }
+
+  public static boolean roleExists(RangerImpalaPlugin plugin, String roleName) {
+    Set<RangerRole> roleSet = plugin.getRoles().getRangerRoles();
+    if (roleSet == null) return false;
+    return roleSet.stream().anyMatch(r -> r.getName().equals(roleName));
+  }
 }
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 5e224f318..7c98bec3b 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -2561,6 +2561,18 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
       authzCatalog_.removeRole("foo_owner");
     }
 
+    // check ALTER VIEW SET OWNER ROLE should throw an AnalysisException.
+    // if role is removed
+    boolean exceptionThrown = false;
+    try {
+      parseAndAnalyze("alter view functional.alltypes_view set owner role foo_owner",
+          authzCtx_, frontend_);
+    } catch (AnalysisException e) {
+      exceptionThrown = true;
+      assertEquals("Role 'foo_owner' does not exist.", e.getLocalizedMessage());
+    }
+    assertTrue(exceptionThrown);
+
     // Database does not exist.
     authorize("alter view nodb.alltypes_view as select 1")
         .error(alterError("nodb"))
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 e0db29e3a..b9b34c6c0 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
@@ -27,13 +27,13 @@ import org.apache.impala.authorization.ranger.RangerAuthorizationFactory;
 import org.apache.impala.authorization.ranger.RangerCatalogdAuthorizationManager;
 import org.apache.impala.authorization.ranger.RangerImpalaPlugin;
 import org.apache.impala.authorization.ranger.RangerImpalaResourceBuilder;
-import org.apache.impala.catalog.Role;
 import org.apache.impala.catalog.ScalarFunction;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.FrontendTestBase;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.service.Frontend;
+import org.apache.impala.testutil.CatalogServiceTestCatalog;
 import org.apache.impala.testutil.ImpaladTestCatalog;
 import org.apache.impala.thrift.TColumnValue;
 import org.apache.impala.thrift.TDescribeOutputStyle;
@@ -112,6 +112,8 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
   protected final RangerImpalaPlugin rangerImpalaPlugin_;
   protected final RangerRESTClient rangerRestClient_;
 
+  protected final CatalogServiceTestCatalog testCatalog_;
+
   public AuthorizationTestBase(AuthorizationProvider authzProvider)
       throws ImpalaException {
     authzProvider_ = authzProvider;
@@ -122,7 +124,8 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
             SERVER_NAME, null, null, null);
         authzFactory_ = createAuthorizationFactory(authzProvider);
         authzCtx_ = createAnalysisCtx(authzFactory_, user_.getName());
-        authzCatalog_ = new ImpaladTestCatalog(authzFactory_);
+        testCatalog_ = CatalogServiceTestCatalogWithRanger.createWithAuth(authzFactory_);
+        authzCatalog_ = new ImpaladTestCatalog(testCatalog_);
         authzFrontend_ = new Frontend(authzFactory_, authzCatalog_);
         rangerImpalaPlugin_ =
             ((RangerAuthorizationChecker) authzFrontend_.getAuthzChecker())
diff --git a/fe/src/test/java/org/apache/impala/authorization/CatalogServiceTestCatalogWithRanger.java b/fe/src/test/java/org/apache/impala/authorization/CatalogServiceTestCatalogWithRanger.java
new file mode 100644
index 000000000..bd3d28888
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/authorization/CatalogServiceTestCatalogWithRanger.java
@@ -0,0 +1,104 @@
+/*
+ * 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.impala.authorization;
+
+import org.apache.impala.authorization.ranger.RangerAuthorizationChecker;
+import org.apache.impala.authorization.ranger.RangerImpalaPlugin;
+import org.apache.impala.authorization.ranger.RangerUtil;
+import org.apache.impala.catalog.MetaStoreClientPool;
+import org.apache.impala.catalog.Role;
+import org.apache.impala.common.ImpalaException;
+import org.apache.impala.testutil.CatalogServiceTestCatalog;
+import org.apache.ranger.plugin.model.RangerRole;
+
+import java.util.List;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class CatalogServiceTestCatalogWithRanger extends CatalogServiceTestCatalog {
+  private static final String RANGER_ADMIN_USER = "admin";
+  private RangerImpalaPlugin rangerImpalaPlugin_;
+  protected CatalogServiceTestCatalogWithRanger(boolean loadInBackground,
+      int numLoadingThreads, MetaStoreClientPool metaStoreClientPool)
+      throws ImpalaException {
+    super(loadInBackground, numLoadingThreads, metaStoreClientPool);
+  }
+
+  public static CatalogServiceTestCatalog create() {
+    return createWithAuth(new NoopAuthorizationFactory());
+  }
+
+  public static CatalogServiceTestCatalog createWithAuth(AuthorizationFactory factory) {
+    return createWithAuth(factory, new BaseTestCatalogSupplier() {
+      @Override
+      public CatalogServiceTestCatalog get() throws ImpalaException {
+        CatalogServiceTestCatalogWithRanger cs;
+        MetaStoreClientPool metaStoreClientPool = new MetaStoreClientPool(0, 0);
+        cs = new CatalogServiceTestCatalogWithRanger(false, 16, metaStoreClientPool);
+        RangerImpalaPlugin rangerImpalaPlugin =
+            ((RangerAuthorizationChecker) factory.newAuthorizationChecker())
+                .getRangerImpalaPlugin();
+        cs.setRangerImpalaPlugin(rangerImpalaPlugin);
+        return cs;
+      }
+    });
+  }
+
+  public void setRangerImpalaPlugin(RangerImpalaPlugin rangerImpalaPlugin_) {
+    this.rangerImpalaPlugin_ = rangerImpalaPlugin_;
+  }
+
+  @Override
+  public Role addRole(String roleName, Set<String> grantGroups) {
+    Role authRole = null;
+    RangerRole role = new RangerRole();
+    role.setName(roleName);
+    role.setCreatedByUser(RANGER_ADMIN_USER);
+    List<RangerRole.RoleMember> roleMemberList =
+        grantGroups.stream()
+            .map(s -> new RangerRole.RoleMember(s, s.equals(RANGER_ADMIN_USER)))
+            .collect(Collectors.toList());
+    role.setGroups(roleMemberList);
+    try {
+      rangerImpalaPlugin_.createRole(role, null);
+      rangerImpalaPlugin_.refreshPoliciesAndTags();
+      authRole = super.addRole(roleName, grantGroups);
+    } catch (Exception ex) {
+      ex.printStackTrace();
+    }
+    return authRole;
+  }
+
+  @Override
+  public Role removeRole(String roleName) {
+    Role authRole = null;
+    try {
+      RangerUtil.validateRangerAdmin(rangerImpalaPlugin_, RANGER_ADMIN_USER);
+      rangerImpalaPlugin_.dropRole(RANGER_ADMIN_USER, roleName, null);
+      // need to invoke plugin to sync policy to avoid stale roles
+      // still exists in plugin cache
+      rangerImpalaPlugin_.refreshPoliciesAndTags();
+      authRole = super.removeRole(roleName);
+    } catch (Exception ex) {
+      ex.printStackTrace();
+    }
+    return authRole;
+  }
+}
diff --git a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java b/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
index 9983fc644..a51aed0d8 100644
--- a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
+++ b/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
@@ -440,6 +440,11 @@ public class FrontendTestBase extends AbstractFrontendTest {
           public void postAnalyze(AuthorizationContext authzCtx) {
           }
 
+          @Override
+          public boolean roleExists(String roleName) {
+            return catalog_.getAuthPolicy().getRole(roleName) != null;
+          }
+
           @Override
           public AuthorizationContext createAuthorizationContext(boolean doAudits,
               String sqlStmt, TSessionState sessionState,
diff --git a/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java b/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
index cba20cbb7..e91678b3a 100644
--- a/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
+++ b/fe/src/test/java/org/apache/impala/testutil/CatalogServiceTestCatalog.java
@@ -45,7 +45,7 @@ import java.util.UUID;
  */
 public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
   private CatalogOpExecutor opExecutor_;
-  private CatalogServiceTestCatalog(boolean loadInBackground, int numLoadingThreads,
+  protected CatalogServiceTestCatalog(boolean loadInBackground, int numLoadingThreads,
       MetaStoreClientPool metaStoreClientPool) throws ImpalaException {
     super(loadInBackground, numLoadingThreads, System.getProperty("java.io.tmpdir"),
         metaStoreClientPool);
@@ -57,6 +57,10 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
     rd.run();
   }
 
+  public interface BaseTestCatalogSupplier {
+    public abstract CatalogServiceTestCatalog get() throws ImpalaException;
+  }
+
   public static CatalogServiceTestCatalog create() {
     return createWithAuth(new NoopAuthorizationFactory());
   }
@@ -66,13 +70,19 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
    * authorization config.
    */
   public static CatalogServiceTestCatalog createWithAuth(AuthorizationFactory factory) {
+    return createWithAuth(factory,
+        () -> new CatalogServiceTestCatalog(false, 16, new MetaStoreClientPool(0, 0)));
+  }
+
+  public static CatalogServiceTestCatalog createWithAuth(
+      AuthorizationFactory factory, BaseTestCatalogSupplier catalogSupplier) {
     FeSupport.loadLibrary();
     CatalogServiceTestCatalog cs;
     try {
       if (MetastoreShim.getMajorVersion() > 2) {
         MetastoreShim.setHiveClientCapabilities();
       }
-      cs = new CatalogServiceTestCatalog(false, 16, new MetaStoreClientPool(0, 0));
+      cs = catalogSupplier.get();
       cs.setAuthzManager(factory.newAuthorizationManager(cs));
       cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
       cs.setCatalogMetastoreServer(NoOpCatalogMetastoreServer.INSTANCE);
@@ -80,15 +90,14 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
           new NoopAuthorizationFactory().getAuthorizationConfig(),
           new NoopAuthorizationFactory.NoopAuthorizationManager(),
           new TestHiveJavaFunctionFactory()));
-      cs.setEventFactoryForSyncToLatestEvent(new EventFactoryForSyncToLatestEvent(
-          cs.getCatalogOpExecutor()));
+      cs.setEventFactoryForSyncToLatestEvent(
+          new EventFactoryForSyncToLatestEvent(cs.getCatalogOpExecutor()));
       cs.reset();
     } catch (ImpalaException e) {
       throw new IllegalStateException(e.getMessage(), e);
     }
     return cs;
   }
-
   /**
    * Creates a transient test catalog instance backed by an embedded HMS derby database on
    * the local filesystem. The derby database is created from scratch and has no table
@@ -119,7 +128,7 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
   @Override
   public AuthorizationPolicy getAuthPolicy() { return authPolicy_; }
 
-  private void setCatalogOpExecutor(CatalogOpExecutor opExecutor) {
+  protected void setCatalogOpExecutor(CatalogOpExecutor opExecutor) {
     opExecutor_ = opExecutor;
   }
 
diff --git a/tests/authorization/test_ranger.py b/tests/authorization/test_ranger.py
index 33cf061b5..db117004f 100644
--- a/tests/authorization/test_ranger.py
+++ b/tests/authorization/test_ranger.py
@@ -1290,6 +1290,7 @@ class TestRanger(CustomClusterTestSuite):
     """Tests ownership privileges for databases and tables with ranger along with
     some known quirks in the implementation."""
     test_user = getuser()
+    test_role = 'test_role'
     test_db = "test_ranger_ownership_" + get_random_id(5).lower()
     # Create a test database as "admin" user. Owner is set accordingly.
     self._run_query_as_user("create database {0}".format(test_db), ADMIN, True)
@@ -1335,6 +1336,18 @@ class TestRanger(CustomClusterTestSuite):
       # Change the table owner back to admin.
       self._run_query_as_user(
           "alter table {0}.foo set owner user {1}".format(test_db, ADMIN), ADMIN, True)
+      # create role before test begin.
+      self._run_query_as_user("CREATE ROLE {0}".format(test_role), ADMIN, True)
+      # test alter table owner to role statement, expect success result.
+      stmt = "alter table {0}.foo set owner role {1}".format(test_db, test_role)
+      self._run_query_as_user(stmt, ADMIN, True)
+      # drop the role.
+      self._run_query_as_user("DROP ROLE {0}".format(test_role), ADMIN, True)
+      # alter table to a non-exist role, expect error showing "role doesn't exist".
+      stmt = "alter table {0}.foo set owner role {1}".format(test_db, test_role)
+      result = self._run_query_as_user(stmt, ADMIN, False)
+      err = "Role '{0}' does not exist.".format(test_role)
+      assert err in str(result)
       # test_user should not be authorized to run the queries anymore.
       result = self._run_query_as_user(
           "select * from {0}.foo".format(test_db), test_user, False)