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

[impala] 02/02: IMPALA-8528: Refactor authorization check in AnalysisContext

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

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

commit 5a23bacdba9f199948b6a971aebca30586c360a5
Author: Fredy Wijaya <fw...@cloudera.com>
AuthorDate: Wed May 8 14:03:52 2019 -0700

    IMPALA-8528: Refactor authorization check in AnalysisContext
    
    This patch moves the authorization check logic from AnalysisContext
    into BaseAuthorizationChecker to consolidate the logic into a single
    place. This patch also converts AuthorizationChecker into an interface
    The existing implementation code of AuthorizationChecker is now moved to
    BaseAuthorizationChecker.
    
    This patch has no functionality change.
    
    Testing:
    - Ran FE tests
    - Ran E2E authorization tests
    
    Change-Id: I3bc3a11220dae0f49ef3e73d9ff27a90e9d4a71c
    Reviewed-on: http://gerrit.cloudera.org:8080/13285
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../apache/impala/analysis/AnalysisContext.java    | 180 +-----------
 .../impala/authorization/AuthorizationChecker.java | 110 +------
 .../authorization/BaseAuthorizationChecker.java    | 319 +++++++++++++++++++++
 .../authorization/NoopAuthorizationFactory.java    |   4 +-
 .../ranger/RangerAuthorizationChecker.java         |   5 +-
 .../sentry/SentryAuthorizationChecker.java         |   6 +-
 .../org/apache/impala/common/FrontendTestBase.java |   5 +-
 7 files changed, 346 insertions(+), 283 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index 64ea780..90eee80 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -20,25 +20,18 @@ package org.apache.impala.analysis;
 import static org.apache.impala.analysis.ToSqlOptions.REWRITTEN;
 
 import java.util.ArrayList;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import org.apache.impala.analysis.StmtMetadataLoader.StmtTableCache;
-import org.apache.impala.authorization.Authorizable;
 import org.apache.impala.authorization.AuthorizationChecker;
 import org.apache.impala.authorization.AuthorizationFactory;
-import org.apache.impala.authorization.Privilege;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.catalog.FeCatalog;
-import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.ImpalaException;
-import org.apache.impala.common.InternalException;
-import org.apache.impala.common.Pair;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.rewrite.ExprRewriter;
 import org.apache.impala.thrift.TAccessEvent;
@@ -50,7 +43,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -426,7 +418,7 @@ public class AnalysisContext {
     // collected during analysis.
     AuthorizationException authException = null;
     try {
-      authorize(authzChecker);
+      authzChecker.authorize(analysisResult_, catalog_);
     } catch (AuthorizationException e) {
       authException = e;
     }
@@ -513,176 +505,6 @@ public class AnalysisContext {
     Preconditions.checkState(!analysisResult_.requiresSubqueryRewrite());
   }
 
-  /**
-   * Authorize an analyzed statement.
-   * analyze() must have already been called. Throws an AuthorizationException if the
-   * user doesn't have sufficient privileges to run this statement.
-   */
-  private void authorize(AuthorizationChecker authzChecker)
-      throws AuthorizationException, InternalException {
-    Preconditions.checkNotNull(analysisResult_);
-    Analyzer analyzer = getAnalyzer();
-    // Authorize statements that may produce several hierarchical privilege requests.
-    // Such a statement always has a corresponding table-level privilege request if it
-    // has column-level privilege request. The hierarchical nature requires special
-    // logic to process correctly and efficiently.
-    if (analysisResult_.isHierarchicalAuthStmt()) {
-      // Map of table name to a list of privilege requests associated with that table.
-      // These include both table-level and column-level privilege requests. We use a
-      // LinkedHashMap to preserve the order in which requests are inserted.
-      Map<String, List<PrivilegeRequest>> tablePrivReqs = new LinkedHashMap<>();
-      // Privilege requests that are not column or table-level.
-      List<PrivilegeRequest> otherPrivReqs = new ArrayList<>();
-      // Group the registered privilege requests based on the table they reference.
-      for (PrivilegeRequest privReq: analyzer.getPrivilegeReqs()) {
-        String tableName = privReq.getAuthorizable().getFullTableName();
-        if (tableName == null) {
-          otherPrivReqs.add(privReq);
-        } else {
-          List<PrivilegeRequest> requests = tablePrivReqs.get(tableName);
-          if (requests == null) {
-            requests = new ArrayList<>();
-            tablePrivReqs.put(tableName, requests);
-          }
-          // The table-level SELECT must be the first table-level request, and it
-          // must precede all column-level privilege requests.
-          Preconditions.checkState((requests.isEmpty() ||
-              !(privReq.getAuthorizable().getType() == Authorizable.Type.COLUMN)) ||
-              (requests.get(0).getAuthorizable().getType() == Authorizable.Type.TABLE &&
-              requests.get(0).getPrivilege() == Privilege.SELECT));
-          requests.add(privReq);
-        }
-      }
-
-      // Check any non-table, non-column privilege requests first.
-      for (PrivilegeRequest request: otherPrivReqs) {
-        authorizePrivilegeRequest(authzChecker, request);
-      }
-
-      // Authorize table accesses, one table at a time, by considering both table and
-      // column-level privilege requests.
-      for (Map.Entry<String, List<PrivilegeRequest>> entry: tablePrivReqs.entrySet()) {
-        authorizeTableAccess(authzChecker, entry.getValue());
-      }
-    } else {
-      for (PrivilegeRequest privReq: analyzer.getPrivilegeReqs()) {
-        Preconditions.checkState(
-            !(privReq.getAuthorizable().getType() == Authorizable.Type.COLUMN) ||
-            analysisResult_.isSingleColumnPrivStmt());
-        authorizePrivilegeRequest(authzChecker, privReq);
-      }
-    }
-
-    // Check all masked requests. If a masked request has an associated error message,
-    // an AuthorizationException is thrown if authorization fails. Masked requests with no
-    // error message are used to check if the user can access the runtime profile.
-    // These checks don't result in an AuthorizationException but set the
-    // 'user_has_profile_access' flag in queryCtx_.
-    for (Pair<PrivilegeRequest, String> maskedReq: analyzer.getMaskedPrivilegeReqs()) {
-      try {
-        authorizePrivilegeRequest(authzChecker, maskedReq.first);
-      } catch (AuthorizationException e) {
-        analysisResult_.setUserHasProfileAccess(false);
-        if (!Strings.isNullOrEmpty(maskedReq.second)) {
-          throw new AuthorizationException(maskedReq.second);
-        }
-        break;
-      }
-    }
-  }
-
-  /**
-   * Authorize a privilege request.
-   * Throws an AuthorizationException if the user doesn't have sufficient privileges for
-   * this request. Also, checks if the request references a system database.
-   */
-  private void authorizePrivilegeRequest(AuthorizationChecker authzChecker,
-      PrivilegeRequest request) throws AuthorizationException, InternalException {
-    Preconditions.checkNotNull(request);
-    String dbName = null;
-    if (request.getAuthorizable() != null) {
-      dbName = request.getAuthorizable().getDbName();
-    }
-    // If this is a system database, some actions should always be allowed
-    // or disabled, regardless of what is in the auth policy.
-    if (dbName != null && checkSystemDbAccess(dbName, request.getPrivilege())) {
-      return;
-    }
-    authzChecker.checkAccess(getAnalyzer().getUser(), request);
-  }
-
-  /**
-   * Authorize a list of privilege requests associated with a single table.
-   * It checks if the user has sufficient table-level privileges and if that is
-   * not the case, it falls back on checking column-level privileges, if any. This
-   * function requires 'SELECT' requests to be ordered by table and then by column
-   * privilege requests. Throws an AuthorizationException if the user doesn't have
-   * sufficient privileges.
-   */
-  private void authorizeTableAccess(AuthorizationChecker authzChecker,
-      List<PrivilegeRequest> requests) throws AuthorizationException, InternalException {
-    Preconditions.checkState(!requests.isEmpty());
-    Analyzer analyzer = getAnalyzer();
-    // We need to temporarily deny access when column masking or row filtering feature is
-    // enabled until Impala has full implementation of column masking and row filtering.
-    // This is to prevent data leak since we do not want Impala to show any information
-    // when Hive has column masking and row filtering enabled.
-    authzChecker.authorizeRowFilterAndColumnMask(getAnalyzer().getUser(), requests);
-
-    boolean hasTableSelectPriv = true;
-    boolean hasColumnSelectPriv = false;
-    for (PrivilegeRequest request: requests) {
-      if (request.getAuthorizable().getType() == Authorizable.Type.TABLE) {
-        try {
-          authorizePrivilegeRequest(authzChecker, request);
-        } catch (AuthorizationException e) {
-          // Authorization fails if we fail to authorize any table-level request that is
-          // not a SELECT privilege (e.g. INSERT).
-          if (request.getPrivilege() != Privilege.SELECT) throw e;
-          hasTableSelectPriv = false;
-        }
-      } else {
-        Preconditions.checkState(
-            request.getAuthorizable().getType() == Authorizable.Type.COLUMN);
-        if (hasTableSelectPriv) continue;
-        if (authzChecker.hasAccess(analyzer.getUser(), request)) {
-          hasColumnSelectPriv = true;
-          continue;
-        }
-        // Make sure we don't reveal any column names in the error message.
-        throw new AuthorizationException(String.format("User '%s' does not have " +
-          "privileges to execute '%s' on: %s", analyzer.getUser().getName(),
-          request.getPrivilege().toString(),
-          request.getAuthorizable().getFullTableName()));
-      }
-    }
-    if (!hasTableSelectPriv && !hasColumnSelectPriv) {
-       throw new AuthorizationException(String.format("User '%s' does not have " +
-          "privileges to execute 'SELECT' on: %s", analyzer.getUser().getName(),
-          requests.get(0).getAuthorizable().getFullTableName()));
-    }
-  }
-
-  /**
-   * Throws an AuthorizationException if the dbName is a system db
-   * and the user is trying to modify it.
-   * Returns true if this is a system db and the action is allowed.
-   */
-  private boolean checkSystemDbAccess(String dbName, Privilege privilege)
-      throws AuthorizationException {
-    FeDb db = catalog_.getDb(dbName);
-    if (db != null && db.isSystemDb()) {
-      switch (privilege) {
-        case VIEW_METADATA:
-        case ANY:
-          return true;
-        default:
-          throw new AuthorizationException("Cannot modify system database.");
-      }
-    }
-    return false;
-  }
-
   public Analyzer getAnalyzer() { return analysisResult_.getAnalyzer(); }
   public EventSequence getTimeline() { return timeline_; }
 }
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 7d643cb..5d4b1be 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizationChecker.java
@@ -17,119 +17,39 @@
 
 package org.apache.impala.authorization;
 
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.impala.authorization.Authorizable.Type;
-
-import com.google.common.base.Preconditions;
+import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
+import org.apache.impala.catalog.FeCatalog;
 import org.apache.impala.common.InternalException;
 
+import java.util.Set;
+
 /**
- * A base class used to check whether a user has access to a given resource.
+ * An interface used to check whether a user has access to a given resource.
  */
-public abstract class AuthorizationChecker {
-  protected final AuthorizationConfig config_;
-
-  /*
-   * Creates a new AuthorizationChecker based on the config values.
-   */
-  protected AuthorizationChecker(AuthorizationConfig config) {
-    Preconditions.checkNotNull(config);
-    config_ = config;
-  }
-
-  /**
-   * Authorizes the PrivilegeRequest, throwing an Authorization exception if
-   * the user does not have sufficient privileges.
-   */
-  public void checkAccess(User user, PrivilegeRequest privilegeRequest)
-      throws AuthorizationException, InternalException {
-    Preconditions.checkNotNull(privilegeRequest);
-
-    if (hasAccess(user, privilegeRequest)) return;
-
-    Privilege privilege = privilegeRequest.getPrivilege();
-    if (privilegeRequest.getAuthorizable().getType() == Type.FUNCTION) {
-      throw new AuthorizationException(String.format(
-          "User '%s' does not have privileges%s to %s functions in: %s",
-          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
-          privilegeRequest.getName()));
-    }
-
-    if (EnumSet.of(Privilege.ANY, Privilege.ALL, Privilege.VIEW_METADATA)
-        .contains(privilege)) {
-      throw new AuthorizationException(String.format(
-          "User '%s' does not have privileges%s to access: %s",
-          user.getName(), grantOption(privilegeRequest.hasGrantOption()),
-          privilegeRequest.getName()));
-    } else if (privilege == Privilege.REFRESH) {
-      throw new AuthorizationException(String.format(
-          "User '%s' does not have privileges%s to execute " +
-          "'INVALIDATE METADATA/REFRESH' on: %s", user.getName(),
-          grantOption(privilegeRequest.hasGrantOption()), privilegeRequest.getName()));
-    } else if (privilege == Privilege.CREATE &&
-        privilegeRequest.getAuthorizable().getType() == Type.TABLE) {
-      // Creating a table requires CREATE on a database and we shouldn't
-      // expose the table name.
-      throw new AuthorizationException(String.format(
-          "User '%s' does not have privileges%s to execute '%s' on: %s",
-          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
-          privilegeRequest.getAuthorizable().getDbName()));
-    } else {
-      throw new AuthorizationException(String.format(
-          "User '%s' does not have privileges%s to execute '%s' on: %s",
-          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
-          privilegeRequest.getName()));
-    }
-  }
-
-  private static String grantOption(boolean hasGrantOption) {
-    return hasGrantOption ? " with 'GRANT OPTION'" : "";
-  }
-
+public interface AuthorizationChecker {
   /*
    * Returns true if the given user has permission to execute the given
    * request, false otherwise. Always returns true if authorization is disabled or the
    * given user is an admin user.
    */
-  public boolean hasAccess(User user, PrivilegeRequest request)
-      throws InternalException {
-    Preconditions.checkNotNull(user);
-    Preconditions.checkNotNull(request);
-
-    // If authorization is not enabled the user will always have access. If this is
-    // an internal request, the user will always have permission.
-    if (!config_.isEnabled() || user instanceof ImpalaInternalAdminUser) {
-      return true;
-    }
-    return authorize(user, request);
-  }
+  boolean hasAccess(User user, PrivilegeRequest request) throws InternalException;
 
   /**
-   * Performs an authorization for a given user.
+   * Authorize an analyzed statement.
+   *
+   * @throws AuthorizationException thrown if the user doesn't have sufficient privileges
+   *                                to run this statement.
    */
-  protected abstract boolean authorize(User user, PrivilegeRequest request)
-      throws InternalException;
+  void authorize(AnalysisResult analysisResult, FeCatalog catalog)
+      throws AuthorizationException, InternalException;
 
   /**
    * Returns a set of groups for a given user.
    */
-  public abstract Set<String> getUserGroups(User user) throws InternalException;
-
-  /**
-   * Checks if the given tables/columns are configured with row filtering/column masking
-   * enabled. If they do, throw an {@link AuthorizationException} to prevent data leak.
-   *
-   * TODO: Remove this method when Impala supports row filtering and column masking.
-   */
-  public abstract void authorizeRowFilterAndColumnMask(User user,
-      List<PrivilegeRequest> privilegeRequests)
-      throws AuthorizationException, InternalException;
+  Set<String> getUserGroups(User user) throws InternalException;
 
   /**
    * Invalidates an authorization cache.
    */
-  public abstract void invalidateAuthorizationCache();
+  void invalidateAuthorizationCache();
 }
diff --git a/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
new file mode 100644
index 0000000..b351307
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
@@ -0,0 +1,319 @@
+// 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 com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
+import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
+import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.authorization.Authorizable.Type;
+import org.apache.impala.catalog.FeCatalog;
+import org.apache.impala.catalog.FeDb;
+import org.apache.impala.common.InternalException;
+import org.apache.impala.common.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A base class for the {@link AuthorizationChecker}.
+ */
+public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
+  private final static Logger LOG = LoggerFactory.getLogger(
+      BaseAuthorizationChecker.class);
+
+  protected final AuthorizationConfig config_;
+
+  /*
+   * Creates a new AuthorizationChecker based on the config values.
+   */
+  protected BaseAuthorizationChecker(AuthorizationConfig config) {
+    Preconditions.checkNotNull(config);
+    config_ = config;
+  }
+
+  /*
+   * Returns true if the given user has permission to execute the given
+   * request, false otherwise. Always returns true if authorization is disabled or the
+   * given user is an admin user.
+   */
+  public boolean hasAccess(User user, PrivilegeRequest request)
+      throws InternalException {
+    Preconditions.checkNotNull(user);
+    Preconditions.checkNotNull(request);
+
+    // If authorization is not enabled the user will always have access. If this is
+    // an internal request, the user will always have permission.
+    if (!config_.isEnabled() || user instanceof ImpalaInternalAdminUser) {
+      return true;
+    }
+    return authorize(user, request);
+  }
+
+  /**
+   * Authorize an analyzed statement.
+   * analyze() must have already been called. Throws an AuthorizationException if the
+   * user doesn't have sufficient privileges to run this statement.
+   */
+  public void authorize(AnalysisResult analysisResult, FeCatalog catalog)
+      throws AuthorizationException, InternalException {
+    Preconditions.checkNotNull(analysisResult);
+    Analyzer analyzer = analysisResult.getAnalyzer();
+    // Authorize statements that may produce several hierarchical privilege requests.
+    // Such a statement always has a corresponding table-level privilege request if it
+    // has column-level privilege request. The hierarchical nature requires special
+    // logic to process correctly and efficiently.
+    if (analysisResult.isHierarchicalAuthStmt()) {
+      // Map of table name to a list of privilege requests associated with that table.
+      // These include both table-level and column-level privilege requests. We use a
+      // LinkedHashMap to preserve the order in which requests are inserted.
+      Map<String, List<PrivilegeRequest>> tablePrivReqs = new LinkedHashMap<>();
+      // Privilege requests that are not column or table-level.
+      List<PrivilegeRequest> otherPrivReqs = new ArrayList<>();
+      // Group the registered privilege requests based on the table they reference.
+      for (PrivilegeRequest privReq : analyzer.getPrivilegeReqs()) {
+        String tableName = privReq.getAuthorizable().getFullTableName();
+        if (tableName == null) {
+          otherPrivReqs.add(privReq);
+        } else {
+          List<PrivilegeRequest> requests = tablePrivReqs.get(tableName);
+          if (requests == null) {
+            requests = new ArrayList<>();
+            tablePrivReqs.put(tableName, requests);
+          }
+          // The table-level SELECT must be the first table-level request, and it
+          // must precede all column-level privilege requests.
+          Preconditions.checkState((requests.isEmpty() ||
+              !(privReq.getAuthorizable().getType() == Authorizable.Type.COLUMN)) ||
+              (requests.get(0).getAuthorizable().getType() == Authorizable.Type.TABLE &&
+                  requests.get(0).getPrivilege() == Privilege.SELECT));
+          requests.add(privReq);
+        }
+      }
+
+      // Check any non-table, non-column privilege requests first.
+      for (PrivilegeRequest request : otherPrivReqs) {
+        authorizePrivilegeRequest(analysisResult, catalog, request);
+      }
+
+      // Authorize table accesses, one table at a time, by considering both table and
+      // column-level privilege requests.
+      for (Map.Entry<String, List<PrivilegeRequest>> entry : tablePrivReqs.entrySet()) {
+        authorizeTableAccess(analysisResult, catalog, entry.getValue());
+      }
+    } else {
+      for (PrivilegeRequest privReq : analyzer.getPrivilegeReqs()) {
+        Preconditions.checkState(
+            !(privReq.getAuthorizable().getType() == Authorizable.Type.COLUMN) ||
+                analysisResult.isSingleColumnPrivStmt());
+        authorizePrivilegeRequest(analysisResult, catalog, privReq);
+      }
+    }
+
+    // Check all masked requests. If a masked request has an associated error message,
+    // an AuthorizationException is thrown if authorization fails. Masked requests with
+    // no error message are used to check if the user can access the runtime profile.
+    // These checks don't result in an AuthorizationException but set the
+    // 'user_has_profile_access' flag in queryCtx_.
+    for (Pair<PrivilegeRequest, String> maskedReq : analyzer.getMaskedPrivilegeReqs()) {
+      try {
+        authorizePrivilegeRequest(analysisResult, catalog, maskedReq.first);
+      } catch (AuthorizationException e) {
+        analysisResult.setUserHasProfileAccess(false);
+        if (!Strings.isNullOrEmpty(maskedReq.second)) {
+          throw new AuthorizationException(maskedReq.second);
+        }
+        break;
+      }
+    }
+  }
+
+  /**
+   * Authorize a privilege request.
+   * Throws an AuthorizationException if the user doesn't have sufficient privileges for
+   * this request. Also, checks if the request references a system database.
+   */
+  private void authorizePrivilegeRequest(AnalysisResult analysisResult, FeCatalog catalog,
+      PrivilegeRequest request) throws AuthorizationException, InternalException {
+    Preconditions.checkNotNull(request);
+    String dbName = null;
+    if (request.getAuthorizable() != null) {
+      dbName = request.getAuthorizable().getDbName();
+    }
+    // If this is a system database, some actions should always be allowed
+    // or disabled, regardless of what is in the auth policy.
+    if (dbName != null && checkSystemDbAccess(catalog, dbName, request.getPrivilege())) {
+      return;
+    }
+    checkAccess(analysisResult.getAnalyzer().getUser(), request);
+  }
+
+  /**
+   * Authorize a list of privilege requests associated with a single table.
+   * It checks if the user has sufficient table-level privileges and if that is
+   * not the case, it falls back on checking column-level privileges, if any. This
+   * function requires 'SELECT' requests to be ordered by table and then by column
+   * privilege requests. Throws an AuthorizationException if the user doesn't have
+   * sufficient privileges.
+   */
+  private void authorizeTableAccess(AnalysisResult analysisResult, FeCatalog catalog,
+      List<PrivilegeRequest> requests) throws AuthorizationException, InternalException {
+    Preconditions.checkArgument(!requests.isEmpty());
+    Analyzer analyzer = analysisResult.getAnalyzer();
+    // We need to temporarily deny access when column masking or row filtering feature is
+    // enabled until Impala has full implementation of column masking and row filtering.
+    // This is to prevent data leak since we do not want Impala to show any information
+    // when Hive has column masking and row filtering enabled.
+    authorizeRowFilterAndColumnMask(analysisResult.getAnalyzer().getUser(), requests);
+
+    boolean hasTableSelectPriv = true;
+    boolean hasColumnSelectPriv = false;
+    for (PrivilegeRequest request: requests) {
+      if (request.getAuthorizable().getType() == Authorizable.Type.TABLE) {
+        try {
+          authorizePrivilegeRequest(analysisResult, catalog, request);
+        } catch (AuthorizationException e) {
+          // Authorization fails if we fail to authorize any table-level request that is
+          // not a SELECT privilege (e.g. INSERT).
+          if (request.getPrivilege() != Privilege.SELECT) throw e;
+          hasTableSelectPriv = false;
+        }
+      } else {
+        Preconditions.checkState(
+            request.getAuthorizable().getType() == Authorizable.Type.COLUMN);
+        if (hasTableSelectPriv) continue;
+        if (hasAccess(analyzer.getUser(), request)) {
+          hasColumnSelectPriv = true;
+          continue;
+        }
+        // Make sure we don't reveal any column names in the error message.
+        throw new AuthorizationException(String.format("User '%s' does not have " +
+                "privileges to execute '%s' on: %s", analyzer.getUser().getName(),
+            request.getPrivilege().toString(),
+            request.getAuthorizable().getFullTableName()));
+      }
+    }
+    if (!hasTableSelectPriv && !hasColumnSelectPriv) {
+      throw new AuthorizationException(String.format("User '%s' does not have " +
+              "privileges to execute 'SELECT' on: %s", analyzer.getUser().getName(),
+          requests.get(0).getAuthorizable().getFullTableName()));
+    }
+  }
+
+  /**
+   * Throws an AuthorizationException if the dbName is a system db
+   * and the user is trying to modify it.
+   * Returns true if this is a system db and the action is allowed.
+   */
+  private boolean checkSystemDbAccess(FeCatalog catalog, String dbName,
+      Privilege privilege)
+      throws AuthorizationException {
+    FeDb db = catalog.getDb(dbName);
+    if (db != null && db.isSystemDb()) {
+      switch (privilege) {
+        case VIEW_METADATA:
+        case ANY:
+          return true;
+        default:
+          throw new AuthorizationException("Cannot modify system database.");
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Authorizes the PrivilegeRequest, throwing an Authorization exception if
+   * the user does not have sufficient privileges.
+   */
+  private void checkAccess(User user, PrivilegeRequest privilegeRequest)
+      throws AuthorizationException, InternalException {
+    Preconditions.checkNotNull(privilegeRequest);
+
+    if (hasAccess(user, privilegeRequest)) return;
+
+    Privilege privilege = privilegeRequest.getPrivilege();
+    if (privilegeRequest.getAuthorizable().getType() == Type.FUNCTION) {
+      throw new AuthorizationException(String.format(
+          "User '%s' does not have privileges%s to %s functions in: %s",
+          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
+          privilegeRequest.getName()));
+    }
+
+    if (EnumSet.of(Privilege.ANY, Privilege.ALL, Privilege.VIEW_METADATA)
+        .contains(privilege)) {
+      throw new AuthorizationException(String.format(
+          "User '%s' does not have privileges%s to access: %s",
+          user.getName(), grantOption(privilegeRequest.hasGrantOption()),
+          privilegeRequest.getName()));
+    } else if (privilege == Privilege.REFRESH) {
+      throw new AuthorizationException(String.format(
+          "User '%s' does not have privileges%s to execute " +
+              "'INVALIDATE METADATA/REFRESH' on: %s", user.getName(),
+          grantOption(privilegeRequest.hasGrantOption()), privilegeRequest.getName()));
+    } else if (privilege == Privilege.CREATE &&
+        privilegeRequest.getAuthorizable().getType() == Type.TABLE) {
+      // Creating a table requires CREATE on a database and we shouldn't
+      // expose the table name.
+      throw new AuthorizationException(String.format(
+          "User '%s' does not have privileges%s to execute '%s' on: %s",
+          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
+          privilegeRequest.getAuthorizable().getDbName()));
+    } else {
+      throw new AuthorizationException(String.format(
+          "User '%s' does not have privileges%s to execute '%s' on: %s",
+          user.getName(), grantOption(privilegeRequest.hasGrantOption()), privilege,
+          privilegeRequest.getName()));
+    }
+  }
+
+  private static String grantOption(boolean hasGrantOption) {
+    return hasGrantOption ? " with 'GRANT OPTION'" : "";
+  }
+
+  /**
+   * Performs an authorization for a given user.
+   */
+  protected abstract boolean authorize(User user, PrivilegeRequest request)
+      throws InternalException;
+
+  /**
+   * Returns a set of groups for a given user.
+   */
+  public abstract Set<String> getUserGroups(User user) throws InternalException;
+
+  /**
+   * Checks if the given tables/columns are configured with row filtering/column masking
+   * enabled. If they do, throw an {@link AuthorizationException} to prevent data leak.
+   *
+   * TODO: Remove this method when Impala supports row filtering and column masking.
+   */
+  protected abstract void authorizeRowFilterAndColumnMask(User user,
+      List<PrivilegeRequest> privilegeRequests)
+      throws AuthorizationException, InternalException;
+
+  /**
+   * Invalidates an authorization cache.
+   */
+  public abstract void invalidateAuthorizationCache();
+}
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 ed77fe1..7ac58aa 100644
--- a/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java
+++ b/fe/src/main/java/org/apache/impala/authorization/NoopAuthorizationFactory.java
@@ -182,7 +182,7 @@ public class NoopAuthorizationFactory implements AuthorizationFactory {
 
   @Override
   public AuthorizationChecker newAuthorizationChecker(AuthorizationPolicy authzPolicy) {
-    return new AuthorizationChecker(authzConfig_) {
+    return new BaseAuthorizationChecker(authzConfig_) {
       @Override
       protected boolean authorize(User user, PrivilegeRequest request)
           throws InternalException {
@@ -195,7 +195,7 @@ public class NoopAuthorizationFactory implements AuthorizationFactory {
       }
 
       @Override
-      public void authorizeRowFilterAndColumnMask(User user,
+      protected void authorizeRowFilterAndColumnMask(User user,
           List<PrivilegeRequest> privilegeRequests)
           throws AuthorizationException, InternalException {
       }
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 b938bee..a985ddf 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
@@ -25,6 +25,7 @@ import org.apache.impala.authorization.Authorizable.Type;
 import org.apache.impala.authorization.AuthorizationChecker;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.AuthorizationException;
+import org.apache.impala.authorization.BaseAuthorizationChecker;
 import org.apache.impala.authorization.DefaultAuthorizableFactory;
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.authorization.PrivilegeRequest;
@@ -51,7 +52,7 @@ import java.util.Set;
  * The Ranger implementation does not use catalog to cache the authorization policy.
  * Ranger plugin uses its own cache.
  */
-public class RangerAuthorizationChecker extends AuthorizationChecker {
+public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
   private static final Logger LOG = LoggerFactory.getLogger(
       RangerAuthorizationChecker.class);
 
@@ -159,7 +160,7 @@ public class RangerAuthorizationChecker extends AuthorizationChecker {
   }
 
   @Override
-  public void authorizeRowFilterAndColumnMask(User user,
+  protected void authorizeRowFilterAndColumnMask(User user,
       List<PrivilegeRequest> privilegeRequests)
       throws AuthorizationException, InternalException {
     for (PrivilegeRequest request : privilegeRequests) {
diff --git a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryAuthorizationChecker.java
index c570600..7b6cb76 100644
--- a/fe/src/main/java/org/apache/impala/authorization/sentry/SentryAuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/sentry/SentryAuthorizationChecker.java
@@ -20,9 +20,9 @@ package org.apache.impala.authorization.sentry;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.impala.authorization.Authorizable.Type;
-import org.apache.impala.authorization.AuthorizationChecker;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.AuthorizationException;
+import org.apache.impala.authorization.BaseAuthorizationChecker;
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.User;
@@ -41,7 +41,7 @@ import java.util.Set;
 /**
  * An implementation of AuthorizationChecker that uses Sentry.
  */
-public class SentryAuthorizationChecker extends AuthorizationChecker {
+public class SentryAuthorizationChecker extends BaseAuthorizationChecker {
   private final ResourceAuthorizationProvider provider_;
   private final SentryAuthorizableServer server_;
 
@@ -75,7 +75,7 @@ public class SentryAuthorizationChecker extends AuthorizationChecker {
   }
 
   @Override
-  public void authorizeRowFilterAndColumnMask(User user,
+  protected void authorizeRowFilterAndColumnMask(User user,
       List<PrivilegeRequest> privilegeRequests)
       throws AuthorizationException, InternalException {
   }
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 8e95f3c..65a93c0 100644
--- a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
+++ b/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
@@ -40,6 +40,7 @@ import org.apache.impala.authorization.AuthorizationException;
 import org.apache.impala.authorization.AuthorizationFactory;
 import org.apache.impala.authorization.AuthorizationManager;
 import org.apache.impala.authorization.AuthorizationPolicy;
+import org.apache.impala.authorization.BaseAuthorizationChecker;
 import org.apache.impala.authorization.NoopAuthorizationFactory.NoopAuthorizationManager;
 import org.apache.impala.authorization.PrivilegeRequest;
 import org.apache.impala.authorization.User;
@@ -333,7 +334,7 @@ public class FrontendTestBase extends AbstractFrontendTest {
       public AuthorizationChecker newAuthorizationChecker(
           AuthorizationPolicy authzPolicy) {
         AuthorizationConfig authzConfig = getAuthorizationConfig();
-        return new AuthorizationChecker(authzConfig) {
+        return new BaseAuthorizationChecker(authzConfig) {
           @Override
           protected boolean authorize(User user, PrivilegeRequest request)
               throws InternalException {
@@ -346,7 +347,7 @@ public class FrontendTestBase extends AbstractFrontendTest {
           }
 
           @Override
-          public void authorizeRowFilterAndColumnMask(User user,
+          protected void authorizeRowFilterAndColumnMask(User user,
               List<PrivilegeRequest> privilegeRequests)
               throws AuthorizationException, InternalException {
           }