You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by lv...@apache.org on 2018/04/25 21:47:47 UTC

[4/7] impala git commit: IMPALA-6643: Add REFRESH fine-grained privilege

IMPALA-6643: Add REFRESH fine-grained privilege

Before this patch, ALL privilege was required to execute INVALIDATE
METADATA and having any privilege allowed executing REFRESH <table>
and INVALIDATE METADATA <table>. With this patch, REFRESH privilege
is now required to execute INVALIDATE METADATA or REFRESH statement.

These are the new GRANT/REVOKE statements introduced at server,
database, and table scopes.

GRANT REFRESH on SERVER svr TO ROLE testrole;
GRANT REFRESH on DATABASE db TO ROLE testrole;
GRANT REFRESH on TABLE db.tbl TO ROLE testrole;

REVOKE REFRESH on SERVER svr FROM ROLE testrole;
REVOKE REFRESH on DATABASE db FROM ROLE testrole;
REVOKE REFRESH on TABLE db.tbl FROM ROLE testrole;

Testing:
- Ran front-end tests

Change-Id: I4c3c5a51fe493d39fd719c7a388d4d5760049ce4
Reviewed-on: http://gerrit.cloudera.org:8080/9589
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/f2a89249
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/f2a89249
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/f2a89249

Branch: refs/heads/2.x
Commit: f2a8924974a5ef970279f6fdf3760804ede650b0
Parents: 3acb8f9
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Mon Mar 12 16:25:47 2018 -0500
Committer: Fredy Wijaya <fw...@cloudera.com>
Committed: Wed Apr 25 13:22:15 2018 -0700

----------------------------------------------------------------------
 common/thrift/CatalogObjects.thrift             |   3 +-
 fe/src/main/cup/sql-parser.cup                  |   2 +
 .../apache/impala/analysis/PrivilegeSpec.java   |   7 +-
 .../impala/analysis/ResetMetadataStmt.java      |  10 +-
 .../authorization/AuthorizationChecker.java     |  11 +-
 .../apache/impala/authorization/Privilege.java  |  52 +++++--
 .../impala/analysis/AnalyzeAuthStmtsTest.java   |  18 ++-
 .../impala/analysis/AuthorizationTest.java      | 137 +++++++++++++++----
 .../org/apache/impala/analysis/ParserTest.java  |   6 +
 fe/src/test/resources/authz-policy.ini.template |   9 +-
 10 files changed, 204 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/common/thrift/CatalogObjects.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/CatalogObjects.thrift b/common/thrift/CatalogObjects.thrift
index 8733af3..badad28 100644
--- a/common/thrift/CatalogObjects.thrift
+++ b/common/thrift/CatalogObjects.thrift
@@ -469,7 +469,8 @@ enum TPrivilegeScope {
 enum TPrivilegeLevel {
   ALL,
   INSERT,
-  SELECT
+  SELECT,
+  REFRESH
 }
 
 // Represents a privilege in an authorization policy. Privileges contain the level

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 46a1d32..4490a8c 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -960,6 +960,8 @@ privilege ::=
   {: RESULT = TPrivilegeLevel.SELECT; :}
   | KW_INSERT
   {: RESULT = TPrivilegeLevel.INSERT; :}
+  | KW_REFRESH
+  {: RESULT = TPrivilegeLevel.REFRESH; :}
   | KW_ALL
   {: RESULT = TPrivilegeLevel.ALL; :}
   ;

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java b/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
index a21af93..cbc3c80 100644
--- a/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
+++ b/fe/src/main/java/org/apache/impala/analysis/PrivilegeSpec.java
@@ -188,9 +188,10 @@ public class PrivilegeSpec implements ParseNode {
 
     switch (scope_) {
       case SERVER:
-        if (privilegeLevel_ != TPrivilegeLevel.ALL) {
-          throw new AnalysisException("Only 'ALL' privilege may be applied at " +
-              "SERVER scope in privilege spec.");
+        if (privilegeLevel_ != TPrivilegeLevel.ALL &&
+            privilegeLevel_ != TPrivilegeLevel.REFRESH) {
+          throw new AnalysisException("Only 'ALL' or 'REFRESH' privilege " +
+              "may be applied at SERVER scope in privilege spec.");
         }
         break;
       case DATABASE:

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/main/java/org/apache/impala/analysis/ResetMetadataStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/ResetMetadataStmt.java b/fe/src/main/java/org/apache/impala/analysis/ResetMetadataStmt.java
index e070d51..a985734 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ResetMetadataStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ResetMetadataStmt.java
@@ -97,7 +97,7 @@ public class ResetMetadataStmt extends StatementBase {
         // Verify the user has privileges to access this table. Will throw if the parent
         // database does not exists. Don't call getTable() to avoid loading the table
         // metadata if it is not yet in this impalad's catalog cache.
-        if (!analyzer.dbContainsTable(dbName, tableName_.getTbl(), Privilege.ANY)) {
+        if (!analyzer.dbContainsTable(dbName, tableName_.getTbl(), Privilege.REFRESH)) {
           // Only throw an exception when the table does not exist for refresh statements
           // since 'invalidate metadata' should add/remove tables created/dropped external
           // to Impala.
@@ -110,10 +110,14 @@ public class ResetMetadataStmt extends StatementBase {
       } else {
         // Verify the user has privileges to access this table.
         analyzer.registerPrivReq(new PrivilegeRequestBuilder()
-            .onTable(dbName, tableName_.getTbl()).any().toRequest());
+            .onTable(dbName, tableName_.getTbl()).allOf(Privilege.REFRESH)
+            .toRequest());
       }
+    } else if (database_ != null) {
+      analyzer.registerPrivReq(new PrivilegeRequestBuilder()
+          .onDb(database_).allOf(Privilege.REFRESH).toRequest());
     } else {
-      analyzer.registerPrivReq(new PrivilegeRequest(Privilege.ALL));
+      analyzer.registerPrivReq(new PrivilegeRequest(Privilege.REFRESH));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
----------------------------------------------------------------------
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 43e86cf..b6fc299 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
@@ -22,12 +22,12 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.reflect.ConstructorUtils;
+import org.apache.impala.authorization.Privilege.SentryAction;
 import org.apache.impala.catalog.AuthorizationException;
 import org.apache.impala.catalog.AuthorizationPolicy;
 import org.apache.impala.common.InternalException;
 import org.apache.sentry.core.common.ActiveRoleSet;
 import org.apache.sentry.core.common.Subject;
-import org.apache.sentry.core.model.db.DBModelAction;
 import org.apache.sentry.core.model.db.DBModelAuthorizable;
 import org.apache.sentry.policy.db.SimpleDBPolicyEngine;
 import org.apache.sentry.provider.cache.SimpleCacheProviderBackend;
@@ -137,6 +137,11 @@ public class AuthorizationChecker {
         throw new AuthorizationException(String.format(
             "User '%s' does not have privileges to access: %s",
             user.getName(), privilegeRequest.getName()));
+      } else if (privilege == Privilege.REFRESH) {
+          throw new AuthorizationException(String.format(
+              "User '%s' does not have privileges to execute " +
+              "'INVALIDATE METADATA/REFRESH' on: %s", user.getName(),
+              privilegeRequest.getName()));
       } else {
         throw new AuthorizationException(String.format(
             "User '%s' does not have privileges to execute '%s' on: %s",
@@ -160,7 +165,7 @@ public class AuthorizationChecker {
       return true;
     }
 
-    EnumSet<DBModelAction> actions = request.getPrivilege().getHiveActions();
+    EnumSet<SentryAction> actions = request.getPrivilege().getSentryActions();
 
     List<DBModelAuthorizable> authorizeables = Lists.newArrayList(
         server_.getHiveAuthorizeableHierarchy());
@@ -172,7 +177,7 @@ public class AuthorizationChecker {
     // The Hive Access API does not currently provide a way to check if the user
     // has any privileges on a given resource.
     if (request.getPrivilege().getAnyOf()) {
-      for (DBModelAction action: actions) {
+      for (SentryAction action: actions) {
         if (provider_.hasAccess(new Subject(user.getShortName()), authorizeables,
             EnumSet.of(action), ActiveRoleSet.ALL)) {
           return true;

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/main/java/org/apache/impala/authorization/Privilege.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/authorization/Privilege.java b/fe/src/main/java/org/apache/impala/authorization/Privilege.java
index 453558b..f82008c 100644
--- a/fe/src/main/java/org/apache/impala/authorization/Privilege.java
+++ b/fe/src/main/java/org/apache/impala/authorization/Privilege.java
@@ -19,45 +19,67 @@ package org.apache.impala.authorization;
 
 import java.util.EnumSet;
 
-import org.apache.sentry.core.model.db.DBModelAction;
+import org.apache.sentry.core.common.Action;
 
 /**
- * Maps an Impala Privilege to one or more Hive Access "Actions".
+ * Maps an Impala Privilege to one or more Sentry "Actions".
  */
 public enum Privilege {
-  ALL(DBModelAction.ALL, false),
-  ALTER(DBModelAction.ALL, false),
-  DROP(DBModelAction.ALL, false),
-  CREATE(DBModelAction.ALL, false),
-  INSERT(DBModelAction.INSERT, false),
-  SELECT(DBModelAction.SELECT, false),
+  ALL(SentryAction.ALL, false),
+  ALTER(SentryAction.ALL, false),
+  DROP(SentryAction.ALL, false),
+  CREATE(SentryAction.ALL, false),
+  INSERT(SentryAction.INSERT, false),
+  SELECT(SentryAction.SELECT, false),
+  REFRESH(SentryAction.REFRESH, false),
   // Privileges required to view metadata on a server object.
-  VIEW_METADATA(EnumSet.of(DBModelAction.INSERT, DBModelAction.SELECT), true),
+  VIEW_METADATA(EnumSet.of(SentryAction.INSERT, SentryAction.SELECT,
+      SentryAction.REFRESH), true),
   // Special privilege that is used to determine if the user has any valid privileges
   // on a target object.
-  ANY(EnumSet.allOf(DBModelAction.class), true),
+  ANY(EnumSet.allOf(SentryAction.class), true),
   ;
 
-  private final EnumSet<DBModelAction> actions;
+  private final EnumSet<SentryAction> actions;
 
   // Determines whether to check if the user has ANY the privileges defined in the
   // actions list or whether to check if the user has ALL of the privileges in the
   // actions list.
   private final boolean anyOf_;
 
-  private Privilege(EnumSet<DBModelAction> actions, boolean anyOf) {
+  /**
+   * This enum provides a list of Sentry actions used in Impala.
+   */
+  public enum SentryAction implements Action {
+    SELECT("select"),
+    INSERT("insert"),
+    REFRESH("refresh"),
+    ALL("*");
+
+    private final String value;
+
+    SentryAction(String value) {
+      this.value = value;
+    }
+
+    public String getValue() {
+      return value;
+    }
+  }
+
+  private Privilege(EnumSet<SentryAction> actions, boolean anyOf) {
     this.actions = actions;
     this.anyOf_ = anyOf;
   }
 
-  private Privilege(DBModelAction action, boolean anyOf) {
+  private Privilege(SentryAction action, boolean anyOf) {
     this(EnumSet.of(action), anyOf);
   }
 
   /*
-   * Returns the set of Hive Access Actions mapping to this Privilege.
+   * Returns the set of Sentry Access Actions mapping to this Privilege.
    */
-  public EnumSet<DBModelAction> getHiveActions() {
+  public EnumSet<SentryAction> getSentryActions() {
     return actions;
   }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
index ddb95f1..f749e1f 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeAuthStmtsTest.java
@@ -165,7 +165,8 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       AnalyzesOk(String.format("%s INSERT ON DATABASE functional %s myrole",
           formatArgs));
       AnalysisError(String.format("%s INSERT ON SERVER %s myrole", formatArgs),
-          "Only 'ALL' privilege may be applied at SERVER scope in privilege spec.");
+          "Only 'ALL' or 'REFRESH' privilege may be applied at SERVER scope " +
+          "in privilege spec.");
       AnalysisError(String.format("%s INSERT ON URI 'hdfs:////abc//123' %s myrole",
           formatArgs), "Only 'ALL' privilege may be applied at URI scope in privilege " +
           "spec.");
@@ -180,7 +181,8 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
       AnalyzesOk(String.format("%s SELECT ON DATABASE functional %s myrole",
           formatArgs));
       AnalysisError(String.format("%s SELECT ON SERVER %s myrole", formatArgs),
-          "Only 'ALL' privilege may be applied at SERVER scope in privilege spec.");
+          "Only 'ALL' or 'REFRESH' privilege may be applied at SERVER scope " +
+          "in privilege spec.");
       AnalysisError(String.format("%s SELECT ON URI 'hdfs:////abc//123' %s myrole",
           formatArgs), "Only 'ALL' privilege may be applied at URI scope in privilege " +
           "spec.");
@@ -219,6 +221,18 @@ public class AnalyzeAuthStmtsTest extends AnalyzerTest {
           "functional.does_not_exist %s myrole", formatArgs), "Error setting " +
           "privileges for table 'functional.does_not_exist'. Verify that the table " +
           "exists and that you have permissions to issue a GRANT/REVOKE statement.");
+
+      // REFRESH privilege
+      AnalyzesOk(String.format(
+          "%s REFRESH ON TABLE functional.alltypes %s myrole", formatArgs));
+      AnalyzesOk(String.format(
+          "%s REFRESH ON DATABASE functional %s myrole", formatArgs));
+      AnalyzesOk(String.format("%s REFRESH ON SERVER %s myrole", formatArgs));
+      AnalyzesOk(String.format("%s REFRESH ON SERVER server1 %s myrole",
+          formatArgs));
+      AnalysisError(String.format(
+          "%s REFRESH ON URI 'hdfs:////abc//123' %s myrole", formatArgs),
+          "Only 'ALL' privilege may be applied at URI scope in privilege spec.");
     }
 
     AnalysisContext authDisabledCtx = createAuthDisabledAnalysisCtx();

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
index db31eaf..a309880 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -80,9 +80,9 @@ public class AuthorizationTest extends FrontendTestBase {
   //   ALL permission on 'tpch' database and 'newdb' database
   //   ALL permission on 'functional_seq_snap' database
   //   SELECT permissions on all tables in 'tpcds' database
-  //   SELECT permissions on 'functional.alltypesagg' (no INSERT permissions)
+  //   SELECT, REFRESH permissions on 'functional.alltypesagg' (no INSERT permissions)
   //   SELECT permissions on 'functional.complex_view' (no INSERT permissions)
-  //   SELECT permissions on 'functional.view_view' (no INSERT permissions)
+  //   SELECT, REFRESH permissions on 'functional.view_view' (no INSERT permissions)
   //   SELECT permissions on columns ('id', 'int_col', and 'year') on
   //   'functional.alltypessmall' (no SELECT permissions on 'functional.alltypessmall')
   //   SELECT permissions on columns ('id', 'int_struct_col', 'struct_array_col',
@@ -96,6 +96,7 @@ public class AuthorizationTest extends FrontendTestBase {
   //   No permissions on database 'functional_rc'
   //   Only column level permissions in 'functional_avro':
   //     SELECT permissions on columns ('id') on 'functional_avro.alltypessmall'
+  //   REFRESH permissions on 'functional_text_lzo' database
   public final static String AUTHZ_POLICY_FILE = "/test-warehouse/authz-policy.ini";
   public final static User USER = new User(System.getProperty("user.name"));
 
@@ -233,6 +234,42 @@ public class AuthorizationTest extends FrontendTestBase {
     privilege.setTable_name(AuthorizeableTable.ANY_TABLE_NAME);
     sentryService.grantRolePrivilege(USER, roleName, privilege);
 
+    // refresh_functional_text_lzo
+    roleName = "refresh_functional_text_lzo";
+    sentryService.createRole(USER, roleName, true);
+    sentryService.grantRoleToGroup(USER, roleName, USER.getName());
+
+    privilege = new TPrivilege("", TPrivilegeLevel.REFRESH,
+        TPrivilegeScope.DATABASE, false);
+    privilege.setServer_name("server1");
+    privilege.setDb_name("functional_text_lzo");
+    privilege.setTable_name(AuthorizeableTable.ANY_TABLE_NAME);
+    sentryService.grantRolePrivilege(USER, roleName, privilege);
+
+    // refresh_functional_alltypesagg
+    roleName = "refresh_functional_alltypesagg";
+    sentryService.createRole(USER, roleName, true);
+    sentryService.grantRoleToGroup(USER, roleName, USER.getName());
+
+    privilege = new TPrivilege("", TPrivilegeLevel.REFRESH,
+        TPrivilegeScope.TABLE, false);
+    privilege.setServer_name("server1");
+    privilege.setDb_name("functional");
+    privilege.setTable_name("alltypesagg");
+    sentryService.grantRolePrivilege(USER, roleName, privilege);
+
+    // refresh_functional_view_view
+    roleName = "refresh_functional_view_view";
+    sentryService.createRole(USER, roleName, true);
+    sentryService.grantRoleToGroup(USER, roleName, USER.getName());
+
+    privilege = new TPrivilege("", TPrivilegeLevel.REFRESH,
+        TPrivilegeScope.TABLE, false);
+    privilege.setServer_name("server1");
+    privilege.setDb_name("functional");
+    privilege.setTable_name("view_view");
+    sentryService.grantRolePrivilege(USER, roleName, privilege);
+
     // all newdb w/ all on URI
     roleName = "all_newdb";
     sentryService.createRole(USER, roleName, true);
@@ -795,45 +832,64 @@ public class AuthorizationTest extends FrontendTestBase {
 
   @Test
   public void TestResetMetadata() throws ImpalaException {
-    // Positive cases (user has privileges on these tables/views).
+    // Positive cases (user has REFRESH privilege on these tables/views).
     AuthzOk("invalidate metadata functional.alltypesagg");
     AuthzOk("refresh functional.alltypesagg");
     AuthzOk("invalidate metadata functional.view_view");
     AuthzOk("refresh functional.view_view");
     // Positive cases for checking refresh partition
     AuthzOk("refresh functional.alltypesagg partition (year=2010, month=1, day=1)");
-    AuthzOk("refresh functional.alltypes partition (year=2009, month=1)");
     AuthzOk("refresh functional_seq_snap.alltypes partition (year=2009, month=1)");
+    // User has REFRESH privilege on functional_text_lzo database, =
+    // but no privilege at the table level.
+    AuthzOk("invalidate metadata functional_text_lzo.alltypes");
+    AuthzOk("refresh functional_text_lzo.alltypes");
+    AuthzOk("refresh functions functional_text_lzo");
 
     AuthzError("invalidate metadata unknown_db.alltypessmall",
-        "User '%s' does not have privileges to access: unknown_db.alltypessmall");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: unknown_db.alltypessmall");
     AuthzError("invalidate metadata functional_seq.alltypessmall",
-        "User '%s' does not have privileges to access: functional_seq.alltypessmall");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional_seq.alltypessmall");
     AuthzError("invalidate metadata functional.alltypes_view",
-        "User '%s' does not have privileges to access: functional.alltypes_view");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypes_view");
     AuthzError("invalidate metadata functional.unknown_table",
-        "User '%s' does not have privileges to access: functional.unknown_table");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.unknown_table");
     AuthzError("invalidate metadata functional.alltypessmall",
-        "User '%s' does not have privileges to access: functional.alltypessmall");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypessmall");
     AuthzError("refresh functional.alltypessmall",
-        "User '%s' does not have privileges to access: functional.alltypessmall");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypessmall");
     AuthzError("refresh functional.alltypes_view",
-        "User '%s' does not have privileges to access: functional.alltypes_view");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypes_view");
     // Only column-level privileges on the table
-    AuthzError("invalidate metadata functional.alltypestiny", "User '%s' does not " +
-        "have privileges to access: functional.alltypestiny");
+    AuthzError("invalidate metadata functional.alltypestiny",
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypestiny");
     // Only column-level privileges on the table
-    AuthzError("refresh functional.alltypestiny", "User '%s' does not have " +
-        "privileges to access: functional.alltypestiny");
+    AuthzError("refresh functional.alltypestiny",
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional.alltypestiny");
 
     AuthzError("invalidate metadata",
-        "User '%s' does not have privileges to access: server");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: server");
     AuthzError(
         "refresh functional_rc.alltypesagg partition (year=2010, month=1, day=1)",
-        "User '%s' does not have privileges to access: functional_rc.alltypesagg");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional_rc.alltypesagg");
     AuthzError(
         "refresh functional_rc.alltypesagg partition (year=2010, month=1, day=9999)",
-        "User '%s' does not have privileges to access: functional_rc.alltypesagg");
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional_rc.alltypesagg");
+    AuthzError("refresh functions functional_rc",
+        "User '%s' does not have privileges to execute 'INVALIDATE METADATA/REFRESH' " +
+        "on: functional_rc");
 
     // TODO: Add test support for dynamically changing privileges for
     // file-based policy.
@@ -842,11 +898,35 @@ public class AuthorizationTest extends FrontendTestBase {
 
     try {
       sentryService.grantRoleToGroup(USER, "admin", USER.getName());
-      ((ImpaladTestCatalog) ctx_.catalog).reset();
+      ctx_.catalog.reset();
       AuthzOk("invalidate metadata");
+      AuthzOk("invalidate metadata functional.testtbl");
+      AuthzOk("refresh functional.testtbl");
+      AuthzOk("refresh functional.alltypesagg partition (year=2010, month=1, day=1)");
+      AuthzOk("refresh functions functional");
     } finally {
       sentryService.revokeRoleFromGroup(USER, "admin", USER.getName());
-      ((ImpaladTestCatalog) ctx_.catalog).reset();
+      ctx_.catalog.reset();
+    }
+
+    // User has REFRESH privilege on server.
+    String roleName = "refresh_role";
+    try {
+      sentryService.createRole(USER, roleName, true);
+      TPrivilege privilege = new TPrivilege("", TPrivilegeLevel.REFRESH,
+          TPrivilegeScope.SERVER, false);
+      privilege.setServer_name("server1");
+      sentryService.grantRolePrivilege(USER, roleName, privilege);
+      sentryService.grantRoleToGroup(USER, roleName, USER.getName());
+      ctx_.catalog.reset();
+      AuthzOk("invalidate metadata");
+      AuthzOk("invalidate metadata functional.testtbl");
+      AuthzOk("refresh functional.testtbl");
+      AuthzOk("refresh functional.alltypesagg partition (year=2010, month=1, day=1)");
+      AuthzOk("refresh functions functional");
+    } finally {
+      sentryService.dropRole(USER, roleName, true);
+      ctx_.catalog.reset();
     }
   }
 
@@ -1443,6 +1523,8 @@ public class AuthorizationTest extends FrontendTestBase {
   @Test
   public void TestDescribeDb() throws ImpalaException {
     AuthzOk("describe database functional_seq_snap");
+    // User has REFRESH privilege on functional_text_lzo database.
+    AuthzOk("describe database functional_text_lzo");
 
     // Database doesn't exist.
     AuthzError("describe database nodb",
@@ -1454,6 +1536,7 @@ public class AuthorizationTest extends FrontendTestBase {
 
   @Test
   public void TestDescribe() throws ImpalaException {
+    // User has SELECT and REFRESH privileges on functional.alltypesagg table.
     AuthzOk("describe functional.alltypesagg");
     AuthzOk("describe functional.alltypes");
     AuthzOk("describe functional.complex_view");
@@ -1542,6 +1625,9 @@ public class AuthorizationTest extends FrontendTestBase {
     AuthzOk("show databases");
     AuthzOk("show tables in _impala_builtins");
     AuthzOk("show functions in _impala_builtins");
+    // User has REFRESH privilege on functional_text_lzo database.
+    AuthzOk("show tables in functional_text_lzo");
+    AuthzOk("show functions in functional_text_lzo");
 
     // Database exists, user does not have access.
     AuthzError("show tables in functional_rc",
@@ -1565,6 +1651,7 @@ public class AuthorizationTest extends FrontendTestBase {
     // Show partitions and show table/column stats.
     String[] statsQuals = new String[] { "partitions", "table stats", "column stats" };
     for (String qual: statsQuals) {
+      // User has SELECT and REFRESH privileges on functional.alltypesagg table.
       AuthzOk(String.format("show %s functional.alltypesagg", qual));
       AuthzOk(String.format("show %s functional.alltypes", qual));
       // User does not have access to db/table.
@@ -1582,6 +1669,8 @@ public class AuthorizationTest extends FrontendTestBase {
     // Show files
     String[] partitions = new String[] { "", "partition(month=10, year=2010)" };
     for (String partition: partitions) {
+      // User has SELECT and REFRESH privileges on functional.alltypesagg table.
+      AuthzOk(String.format("show files in functional.alltypesagg %s", partition));
       AuthzOk(String.format("show files in functional.alltypes %s", partition));
       // User does not have access to db/table.
       AuthzError(String.format("show files in nodb.tbl %s", partition),
@@ -1601,7 +1690,8 @@ public class AuthorizationTest extends FrontendTestBase {
     // These are the only dbs that should show up because they are the only
     // dbs the user has any permissions on.
     List<String> expectedDbs = Lists.newArrayList("default", "functional",
-        "functional_avro", "functional_parquet", "functional_seq_snap", "tpcds", "tpch");
+        "functional_avro", "functional_parquet", "functional_seq_snap",
+        "functional_text_lzo", "tpcds", "tpch");
 
     List<Db> dbs = fe_.getDbs(PatternMatcher.createHivePatternMatcher("*"), USER);
     assertEquals(expectedDbs, extractDbNames(dbs));
@@ -1663,11 +1753,11 @@ public class AuthorizationTest extends FrontendTestBase {
 
   @Test
   public void TestShowCreateTable() throws ImpalaException {
+    // User has SELECT and REFRESH privileges functional.alltypesagg table.
     AuthzOk("show create table functional.alltypesagg");
     AuthzOk("show create table functional.alltypes");
     // Have permissions on view and underlying table.
     AuthzOk("show create table functional_seq_snap.alltypes_view");
-
     // Unqualified table name.
     AuthzError("show create table alltypes",
         "User '%s' does not have privileges to access: default.alltypes");
@@ -1764,7 +1854,8 @@ public class AuthorizationTest extends FrontendTestBase {
     req.get_schemas_req.setSchemaName("%");
     TResultSet resp = fe_.execHiveServer2MetadataOp(req);
     List<String> expectedDbs = Lists.newArrayList("default", "functional",
-        "functional_avro", "functional_parquet", "functional_seq_snap", "tpcds", "tpch");
+        "functional_avro", "functional_parquet", "functional_seq_snap",
+        "functional_text_lzo", "tpcds", "tpch");
     assertEquals(expectedDbs.size(), resp.rows.size());
     for (int i = 0; i < resp.rows.size(); ++i) {
       assertEquals(expectedDbs.get(i),

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index b51d148..aefa8e2 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3575,6 +3575,12 @@ public class ParserTest extends FrontendTestBase {
           formatStr));
       ParserError(String.format("%s SELECT (a, b) ON URI 'foo' %s myRole", formatStr));
 
+      // REFRESH privilege.
+      ParsesOk(String.format("%s REFRESH ON SERVER %s myRole", formatStr));
+      ParsesOk(String.format("%s REFRESH ON SERVER foo %s myRole", formatStr));
+      ParsesOk(String.format("%s REFRESH ON DATABASE foo %s myRole", formatStr));
+      ParsesOk(String.format("%s REFRESH ON TABLE foo %s myRole", formatStr));
+
       // Server scope does not accept a name.
       ParsesOk(String.format("%s ALL ON SERVER %s myRole", formatStr));
       ParsesOk(String.format("%s INSERT ON SERVER %s myRole", formatStr));

http://git-wip-us.apache.org/repos/asf/impala/blob/f2a89249/fe/src/test/resources/authz-policy.ini.template
----------------------------------------------------------------------
diff --git a/fe/src/test/resources/authz-policy.ini.template b/fe/src/test/resources/authz-policy.ini.template
index 18abdb6..f3c7c1f 100644
--- a/fe/src/test/resources/authz-policy.ini.template
+++ b/fe/src/test/resources/authz-policy.ini.template
@@ -24,7 +24,9 @@ ${USER} = all_tpch, all_newdb, all_functional_seq_snap, select_tpcds,\
           select_functional_alltypesagg, insert_functional_alltypes,\
           select_functional_complex_view, select_functional_view_view,\
           insert_parquet, new_table_uri, tpch_data_uri, select_column_level_functional,\
-          select_column_level_functional_avro, upper_case_uri
+          select_column_level_functional_avro, upper_case_uri,\
+          refresh_functional_text_lzo, refresh_functional_alltypesagg,\
+          refresh_functional_view_view
 auth_to_local_group = test_role
 server_admin = all_server
 
@@ -47,6 +49,11 @@ select_functional_complex_view =\
 select_functional_view_view =\
     server=server1->db=functional->table=view_view->action=select
 insert_parquet = server=server1->db=functional_parquet->table=*->action=insert
+refresh_functional_text_lzo = server=server1->db=functional_text_lzo->action=refresh
+refresh_functional_alltypesagg =\
+    server=server1->db=functional->table=alltypesagg->action=refresh
+refresh_functional_view_view =\
+    server=server1->db=functional->table=view_view->action=refresh
 select_column_level_functional =\
     server=server1->db=functional->table=alltypessmall->column=id->action=select,\
     server=server1->db=functional->table=alltypessmall->column=int_col->action=select,\