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 2021/03/31 06:26:32 UTC

[impala] branch master updated: IMPALA-10554: Block updates when row-filter/column-mask is enabled for the user

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


The following commit(s) were added to refs/heads/master by this push:
     new 2baed42  IMPALA-10554: Block updates when row-filter/column-mask is enabled for the user
2baed42 is described below

commit 2baed42736481280d9279642efeb72b504b832ba
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Wed Mar 24 20:55:20 2021 +0800

    IMPALA-10554: Block updates when row-filter/column-mask is enabled for the user
    
    Per RANGER-1087 and RANGER-1100, table updates(e.g. insert, delete,
    truncate, upsert, alter, etc.) should be blocked when row-filtering or
    column-masking policy is enabled for the user.
    
    This patch adds the check for any row-filtering or column-masking policy
    on the table and rejects the update operation if any of them exisits.
    
    Tests:
     - Add FE unit tests
     - Add audit tests
     - Add e2e tests
    
    Change-Id: I1c899f2ec24b895867cbf2cf9ed23bc7b5a77326
    Reviewed-on: http://gerrit.cloudera.org:8080/17230
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../impala/authorization/AuthorizableTable.java    |  13 +
 .../authorization/BaseAuthorizationChecker.java    |  29 +-
 .../org/apache/impala/authorization/Privilege.java |   8 +
 .../ranger/RangerAuthorizationChecker.java         | 118 ++++++--
 .../authorization/ranger/RangerImpalaPlugin.java   |  16 ++
 .../authorization/AuthorizationStmtTest.java       | 297 ++++++++++++++++-----
 .../authorization/AuthorizationTestBase.java       |  19 +-
 .../authorization/ranger/RangerAuditLogTest.java   | 164 +++++++++++-
 .../queries/QueryTest/ranger_column_masking.test   |  18 ++
 .../ranger_column_masking_and_row_filtering.test   |  18 ++
 .../queries/QueryTest/ranger_row_filtering.test    |  18 ++
 11 files changed, 602 insertions(+), 116 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/authorization/AuthorizableTable.java b/fe/src/main/java/org/apache/impala/authorization/AuthorizableTable.java
index 496b579..31d818e 100644
--- a/fe/src/main/java/org/apache/impala/authorization/AuthorizableTable.java
+++ b/fe/src/main/java/org/apache/impala/authorization/AuthorizableTable.java
@@ -22,6 +22,9 @@ import javax.annotation.Nullable;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 
+import java.util.ArrayList;
+import java.util.List;
+
 /**
  * A class to authorize access to a table.
  */
@@ -33,6 +36,7 @@ public class AuthorizableTable extends Authorizable {
   // TODO(IMPALA-10122): Remove the following field once we can properly process a
   // PrivilegeRequest for a view whose creation was not authorized.
   private final boolean viewCreatedWithoutAuthz_;
+  private final List<String> columns_ = new ArrayList<>();
 
   public AuthorizableTable(String dbName, String tableName, @Nullable String ownerUser,
       boolean viewCreatedWithoutAuthz) {
@@ -65,4 +69,13 @@ public class AuthorizableTable extends Authorizable {
   // TODO(IMPALA-10122): Remove the following method once we can properly process a
   // PrivilegeRequest for a view whose creation was not authorized.
   public boolean isViewCreatedWithoutAuthz() { return viewCreatedWithoutAuthz_; }
+
+  public void setColumns(List<String> columns) {
+    columns_.clear();
+    columns_.addAll(columns);
+  }
+
+  public List<String> getColumns() {
+    return columns_;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
index 09bb780..a8c7322 100644
--- a/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
+++ b/fe/src/main/java/org/apache/impala/authorization/BaseAuthorizationChecker.java
@@ -24,6 +24,8 @@ 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.catalog.FeIncompleteTable;
+import org.apache.impala.catalog.FeTable;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.Pair;
 import org.apache.impala.util.EventSequence;
@@ -211,6 +213,22 @@ public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
     if (dbName != null && checkSystemDbAccess(catalog, dbName, request.getPrivilege())) {
       return;
     }
+    // Populate column names to check column masking policies in blocking updates.
+    if (config_.isEnabled() && request.getAuthorizable() != null
+        && request.getAuthorizable().getType() == Type.TABLE) {
+      Preconditions.checkNotNull(dbName);
+      AuthorizableTable authorizableTable = (AuthorizableTable) request.getAuthorizable();
+      FeDb db = catalog.getDb(dbName);
+      if (db != null) {
+        // 'db', 'table' could be null for an unresolved table ref. 'table' could be
+        // null for target table of a CTAS statement. Don't need to populate column
+        // names in such cases since no column masking policies will be checked.
+        FeTable table = db.getTable(authorizableTable.getTableName());
+        if (table != null && !(table instanceof FeIncompleteTable)) {
+          authorizableTable.setColumns(table.getColumnNames());
+        }
+      }
+    }
     checkAccess(authzCtx, analysisResult.getAnalyzer().getUser(), request);
   }
 
@@ -227,12 +245,11 @@ public abstract class BaseAuthorizationChecker implements AuthorizationChecker {
       throws AuthorizationException, InternalException {
     Preconditions.checkArgument(!requests.isEmpty());
     Analyzer analyzer = analysisResult.getAnalyzer();
-    // We need to temporarily deny access when row filtering feature is enabled until
-    // Impala has full implementation of row filtering. Also deny access of columns
-    // containing column masking policies when column masking feature is disabled.
-    // This is to prevent data leak since we do not want Impala to show any information
-    // when Hive has row filtering enabled.
-    authorizeRowFilterAndColumnMask(analysisResult.getAnalyzer().getUser(), requests);
+    // Deny access of columns containing column masking policies when column masking
+    // feature is disabled. Deny access of the tables containing row filtering policies
+    // when row filtering feature is disabled. This is to prevent data leak since we do
+    // not want Impala to show any information when these features are disabled.
+    authorizeRowFilterAndColumnMask(analyzer.getUser(), requests);
 
     boolean hasTableSelectPriv = true;
     boolean hasColumnSelectPriv = false;
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 23c3d78..eb45947 100644
--- a/fe/src/main/java/org/apache/impala/authorization/Privilege.java
+++ b/fe/src/main/java/org/apache/impala/authorization/Privilege.java
@@ -85,4 +85,12 @@ public enum Privilege {
    * Gets list of implied privileges for this privilege.
    */
   public EnumSet<Privilege> getImpliedPrivileges() { return implied_; }
+
+  /**
+   * Returns true if this implies modification on data or metadata.
+   */
+  public boolean impliesUpdate() {
+    return this == ALTER || this == DROP || this == CREATE || this == INSERT
+        || this == REFRESH || this == ALL;
+  }
 }
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 7d66835..c9548a0 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
@@ -19,11 +19,13 @@ package org.apache.impala.authorization.ranger;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
 import org.apache.impala.authorization.Authorizable;
 import org.apache.impala.authorization.Authorizable.Type;
+import org.apache.impala.authorization.AuthorizableTable;
 import org.apache.impala.authorization.AuthorizationChecker;
 import org.apache.impala.authorization.AuthorizationConfig;
 import org.apache.impala.authorization.AuthorizationContext;
@@ -90,6 +92,7 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
     RangerAuthorizationContext rangerAuthzCtx = (RangerAuthorizationContext) authzCtx;
     List<RangerAccessResourceImpl> resources = new ArrayList<>();
     Authorizable authorizable = request.getAuthorizable();
+    Privilege privilege = request.getPrivilege();
     switch (authorizable.getType()) {
       case SERVER:
         // Hive service definition does not have a concept of server. So we define
@@ -128,11 +131,11 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
         // For example any column in foo.bar table:
         // access type: RangerPolicyEngine.ANY_ACCESS
         // resources: [server=server1, database=foo, table=bar]
-        if (request.getPrivilege() != Privilege.ANY ||
+        if (privilege != Privilege.ANY ||
           !DefaultAuthorizableFactory.ALL.equals(authorizable.getTableName())) {
           builder.table(authorizable.getTableName());
         }
-        if (request.getPrivilege() != Privilege.ANY ||
+        if (privilege != Privilege.ANY ||
           !DefaultAuthorizableFactory.ALL.equals(authorizable.getColumnName())) {
           builder.column(authorizable.getColumnName());
         }
@@ -156,15 +159,15 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
     }
 
     for (RangerAccessResourceImpl resource: resources) {
-      if (request.getPrivilege() == Privilege.ANY) {
-        if (!authorizeResource(rangerAuthzCtx, resource, user, request.getPrivilege(),
-            ((RangerAuthorizationContext) authzCtx).getAuditHandler())) {
+      if (privilege == Privilege.ANY) {
+        if (!authorizeResource(rangerAuthzCtx, user, resource, authorizable, privilege,
+            rangerAuthzCtx.getAuditHandler())) {
           return false;
         }
       } else {
-        boolean authorized = request.getPrivilege().hasAnyOf() ?
-            authorizeAny(rangerAuthzCtx, resource, user, request.getPrivilege()) :
-            authorizeAll(rangerAuthzCtx, resource, user, request.getPrivilege());
+        boolean authorized = privilege.hasAnyOf() ?
+            authorizeAny(rangerAuthzCtx, resource, authorizable, user, privilege) :
+            authorizeAll(rangerAuthzCtx, resource, authorizable, user, privilege);
         if (!authorized) {
           return false;
         }
@@ -193,6 +196,7 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
       throws AuthorizationException, InternalException {
     boolean isColumnMaskingEnabled = BackendConfig.INSTANCE.isColumnMaskingEnabled();
     boolean isRowFilteringEnabled = BackendConfig.INSTANCE.isRowFilteringEnabled();
+    if (isColumnMaskingEnabled && isRowFilteringEnabled) return;
     for (PrivilegeRequest request : privilegeRequests) {
       if (!isColumnMaskingEnabled
           && request.getAuthorizable().getType() == Type.COLUMN) {
@@ -400,7 +404,8 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
     } else if (StringUtils.isNotEmpty(transformer)) {
       maskedColumn = transformer.replace("{col}", columnName);
     }
-    LOG.info("dbName: {}, tableName: {}, column: {}, maskType: {}, columnTransformer: {}",
+    LOG.trace(
+        "dbName: {}, tableName: {}, column: {}, maskType: {}, columnTransformer: {}",
         dbName, tableName, columnName, maskType, maskedColumn);
     return maskedColumn;
   }
@@ -420,7 +425,7 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
       return null;
     }
     String filter = accessResult.getFilterExpr();
-    LOG.info("dbName: {}, tableName: {}, rowFilter: {}", dbName, tableName, filter);
+    LOG.trace("dbName: {}, tableName: {}, rowFilter: {}", dbName, tableName, filter);
     return filter;
   }
 
@@ -502,8 +507,8 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
   }
 
   private boolean authorizeAny(RangerAuthorizationContext authzCtx,
-      RangerAccessResourceImpl resource, User user, Privilege privilege)
-      throws InternalException {
+      RangerAccessResourceImpl resource, Authorizable authorizable, User user,
+      Privilege privilege) throws InternalException {
     boolean authorized = false;
     RangerBufferAuditHandler originalAuditHandler = authzCtx.getAuditHandler();
     // Use a temporary audit handler instead of the original audit handler
@@ -515,7 +520,7 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
         (originalAuditHandler == null || !authzCtx.getRetainAudits()) ?
         null : new RangerBufferAuditHandler(originalAuditHandler);
     for (Privilege impliedPrivilege: privilege.getImpliedPrivileges()) {
-      if (authorizeResource(authzCtx, resource, user, impliedPrivilege,
+      if (authorizeResource(authzCtx, user, resource, authorizable, impliedPrivilege,
           tmpAuditHandler)) {
         authorized = true;
         break;
@@ -531,8 +536,8 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
   }
 
   private boolean authorizeAll(RangerAuthorizationContext authzCtx,
-      RangerAccessResourceImpl resource, User user, Privilege privilege)
-      throws InternalException {
+      RangerAccessResourceImpl resource, Authorizable authorizable, User user,
+      Privilege privilege) throws InternalException {
     boolean authorized = true;
     RangerBufferAuditHandler originalAuditHandler = authzCtx.getAuditHandler();
     // Use a temporary audit handler instead of the original audit handler
@@ -547,7 +552,7 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
         (originalAuditHandler == null || !authzCtx.getRetainAudits()) ?
         null : new RangerBufferAuditHandler(originalAuditHandler);
     for (Privilege impliedPrivilege: privilege.getImpliedPrivileges()) {
-      if (!authorizeResource(authzCtx, resource, user, impliedPrivilege,
+      if (!authorizeResource(authzCtx, user, resource, authorizable, impliedPrivilege,
           tmpAuditHandler)) {
         authorized = false;
         break;
@@ -590,8 +595,8 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
     }
   }
 
-  private boolean authorizeResource(RangerAuthorizationContext authzCtx,
-      RangerAccessResourceImpl resource, User user, Privilege privilege,
+  private boolean authorizeResource(RangerAuthorizationContext authzCtx, User user,
+      RangerAccessResourceImpl resource, Authorizable authorizable, Privilege privilege,
       RangerBufferAuditHandler auditHandler) throws InternalException {
     String accessType;
     if (privilege == Privilege.ANY) {
@@ -610,7 +615,82 @@ public class RangerAuthorizationChecker extends BaseAuthorizationChecker {
           authzCtx.getSessionState().getNetwork_address().getHostname());
     }
     RangerAccessResult authorized = plugin_.isAccessAllowed(request, auditHandler);
-    return authorized != null && authorized.getIsAllowed();
+    if (authorized == null || !authorized.getIsAllowed()) return false;
+    // We are done if don't need to block updates on tables that have column-masking or
+    // row-filtering policies.
+    if (!plugin_.blockUpdateIfTableMaskSpecified() || !privilege.impliesUpdate()
+        || (authorizable.getType() != Type.TABLE
+           && authorizable.getType() != Type.COLUMN)) {
+      return true;
+    }
+    return authorizeByTableMasking(request, user, authorizable, authorized, privilege,
+        auditHandler);
+  }
+
+  /**
+   * Blocks the update request if any row-filtering or column masking policy exists on the
+   * resource(table/column). Appends a deny audit if any exists.
+   * Returns true if the request is authorized.
+   */
+  private boolean authorizeByTableMasking(RangerAccessRequestImpl request, User user,
+      Authorizable authorizable, RangerAccessResult accessResult, Privilege privilege,
+      RangerBufferAuditHandler auditHandler) throws InternalException {
+    Preconditions.checkNotNull(accessResult, "accessResult is null!");
+    Preconditions.checkState(accessResult.getIsAllowed(),
+        "update should be allowed before checking this");
+    String originalAccessType = request.getAccessType();
+    // Row-filtering/Column-masking policies are defined only for SELECT requests.
+    request.setAccessType(SELECT_ACCESS_TYPE);
+    // Check if row filtering is enabled for the table/view.
+    if (authorizable.getType() == Type.TABLE) {
+      RangerAccessResult rowFilterResult = plugin_.evalRowFilterPolicies(
+          request, /*resultProcessor*/null);
+      if (rowFilterResult != null && rowFilterResult.isRowFilterEnabled()) {
+        LOG.trace("Deny {} on {} due to row filtering policy {}",
+            privilege, authorizable.getName(), rowFilterResult.getPolicyId());
+        accessResult.setIsAllowed(false);
+        accessResult.setPolicyId(rowFilterResult.getPolicyId());
+        accessResult.setReason("User does not have access to all rows of the table");
+      } else {
+        LOG.trace("No row filtering policy found on {}.", authorizable.getName());
+      }
+    }
+    // Check if masking is enabled for any column in the table/view.
+    if (accessResult.getIsAllowed()) {
+      List<String> columns;
+      if (authorizable.getType() == Type.TABLE) {
+        // Check all columns.
+        columns = ((AuthorizableTable) authorizable).getColumns();
+        LOG.trace("Checking mask policies on {} columns of table {}", columns.size(),
+            authorizable.getFullTableName());
+      } else {
+        columns = Lists.newArrayList(authorizable.getColumnName());
+      }
+      for (String column : columns) {
+        RangerAccessResult columnMaskResult = evalColumnMask(user,
+            authorizable.getDbName(), authorizable.getTableName(), column,
+            /*auditHandler*/null);
+        if (columnMaskResult != null && columnMaskResult.isMaskEnabled()) {
+          LOG.trace("Deny {} on {} due to column masking policy {}",
+              privilege, authorizable.getName(), columnMaskResult.getPolicyId());
+          accessResult.setIsAllowed(false);
+          accessResult.setPolicyId(columnMaskResult.getPolicyId());
+          accessResult.setReason("User does not have access to unmasked column values");
+          break;
+        } else {
+          LOG.trace("No column masking policy found on column {} of {}.", column,
+              authorizable.getFullTableName());
+        }
+      }
+    }
+    // Set back the original access type. The request object is still referenced by the
+    // access result.
+    request.setAccessType(originalAccessType);
+    // Only add deny audits.
+    if (!accessResult.getIsAllowed() && auditHandler != null) {
+      auditHandler.processResult(accessResult);
+    }
+    return accessResult.getIsAllowed();
   }
 
   @VisibleForTesting
diff --git a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
index 5c9b860..102bca3 100644
--- a/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
+++ b/fe/src/main/java/org/apache/impala/authorization/ranger/RangerImpalaPlugin.java
@@ -19,6 +19,7 @@ package org.apache.impala.authorization.ranger;
 
 import com.google.common.base.Preconditions;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.ranger.authorization.hadoop.constants.RangerHadoopConstants;
 import org.apache.ranger.plugin.model.RangerServiceDef;
 import org.apache.ranger.plugin.service.RangerBasePlugin;
 
@@ -38,11 +39,26 @@ public class RangerImpalaPlugin extends RangerBasePlugin {
   private static volatile RangerImpalaPlugin INSTANCE = null;
   private static String SERVICE_TYPE = null;
   private static String APP_ID = null;
+  private static boolean BLOCK_UPDATE_IF_TABLE_MASK_SPECIFIED = true;
 
   private RangerImpalaPlugin(String serviceType, String appId) {
     super(serviceType, appId);
   }
 
+  @Override
+  public void init() {
+    super.init();
+    RangerImpalaPlugin.BLOCK_UPDATE_IF_TABLE_MASK_SPECIFIED = getConfig().getBoolean(
+        RangerHadoopConstants
+            .HIVE_BLOCK_UPDATE_IF_ROWFILTER_COLUMNMASK_SPECIFIED_PROP,
+        RangerHadoopConstants
+            .HIVE_BLOCK_UPDATE_IF_ROWFILTER_COLUMNMASK_SPECIFIED_DEFAULT_VALUE);
+  }
+
+  public boolean blockUpdateIfTableMaskSpecified() {
+    return BLOCK_UPDATE_IF_TABLE_MASK_SPECIFIED;
+  }
+
   public static RangerImpalaPlugin getInstance(String serviceType, String appId) {
     if (INSTANCE == null) {
       synchronized(RangerImpalaPlugin.class) {
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 db8032c..ce2b6be 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -2823,55 +2823,63 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
             onTable("functional", "alltypes", TPrivilegeLevel.ALL));
   }
 
+  private void createColumnMaskingPolicy(String policyName, String dbName, String tblName,
+      String colName, String user, String maskType, String maskExpr) {
+    String json = String.format("{\n" +
+        "  \"name\": \"%s\",\n" +
+        "  \"policyType\": 1,\n" +
+        "  \"serviceType\": \"" + RANGER_SERVICE_TYPE + "\",\n" +
+        "  \"service\": \"" + RANGER_SERVICE_NAME + "\",\n" +
+        "  \"resources\": {\n" +
+        "    \"database\": {\n" +
+        "      \"values\": [\"%s\"],\n" +
+        "      \"isExcludes\": false,\n" +
+        "      \"isRecursive\": false\n" +
+        "    },\n" +
+        "    \"table\": {\n" +
+        "      \"values\": [\"%s\"],\n" +
+        "      \"isExcludes\": false,\n" +
+        "      \"isRecursive\": false\n" +
+        "    },\n" +
+        "    \"column\": {\n" +
+        "      \"values\": [\"%s\"],\n" +
+        "      \"isExcludes\": false,\n" +
+        "      \"isRecursive\": false\n" +
+        "    }\n" +
+        "  },\n" +
+        "  \"dataMaskPolicyItems\": [\n" +
+        "    {\n" +
+        "      \"accesses\": [\n" +
+        "        {\n" +
+        "          \"type\": \"select\",\n" +
+        "          \"isAllowed\": true\n" +
+        "        }\n" +
+        "      ],\n" +
+        "      \"users\": [\"%s\"],\n" +
+        "      \"dataMaskInfo\": {\n" +
+        "        \"dataMaskType\": \"%s\"\n" +
+        "        %s\n" +
+        "      }\n" +
+        "    }\n" +
+        "  ]\n" +
+        "}", policyName, dbName, tblName, colName, user, maskType,
+        maskExpr == null ? "" : String.format(", \"valueExpr\": \"%s\"", maskExpr));
+    createRangerPolicy(policyName, json);
+  }
+
   /**
    * Test the error messages when Column Masking is disabled.
    */
   @Test
-  public void testColumnMaskEnabled() throws ImpalaException {
+  public void testColumnMaskDisabled() throws ImpalaException {
     String policyName = "col_mask";
     for (String tableName: new String[]{"alltypes", "alltypes_view"}) {
       BackendConfig.INSTANCE.setColumnMaskingEnabled(false);
       // Row filtering depends on column masking. So we should disable it as well.
       BackendConfig.INSTANCE.setRowFilteringEnabled(false);
-      String json = String.format("{\n" +
-              "  \"name\": \"%s\",\n" +
-              "  \"policyType\": 1,\n" +
-              "  \"serviceType\": \"%s\",\n" +
-              "  \"service\": \"%s\",\n" +
-              "  \"resources\": {\n" +
-              "    \"database\": {\n" +
-              "      \"values\": [\"functional\"],\n" +
-              "      \"isExcludes\": false,\n" +
-              "      \"isRecursive\": false\n" +
-              "    },\n" +
-              "    \"table\": {\n" +
-              "      \"values\": [\"%s\"],\n" +
-              "      \"isExcludes\": false,\n" +
-              "      \"isRecursive\": false\n" +
-              "    },\n" +
-              "    \"column\": {\n" +
-              "      \"values\": [\"string_col\"],\n" +
-              "      \"isExcludes\": false,\n" +
-              "      \"isRecursive\": false\n" +
-              "    }\n" +
-              "  },\n" +
-              "  \"dataMaskPolicyItems\": [\n" +
-              "    {\n" +
-              "      \"accesses\": [\n" +
-              "        {\n" +
-              "          \"type\": \"select\",\n" +
-              "          \"isAllowed\": true\n" +
-              "        }\n" +
-              "      ],\n" +
-              "      \"users\": [\"%s\"],\n" +
-              "      \"dataMaskInfo\": {\"dataMaskType\": \"MASK\"}\n" +
-              "    }\n" +
-              "  ]\n" +
-              "}", policyName, RANGER_SERVICE_TYPE, RANGER_SERVICE_NAME, tableName,
-          user_.getShortName());
-
       try {
-        createRangerPolicy(policyName, json);
+        createColumnMaskingPolicy(policyName, "functional", tableName, "string_col",
+            user_.getShortName(), "MASK", /*maskExpr*/null);
         rangerImpalaPlugin_.refreshPoliciesAndTags();
 
         // Queries on columns that are not masked should be allowed.
@@ -2949,48 +2957,52 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
     }
   }
 
+  private void createRowFilteringPolicy(String policyName, String dbName, String tblName,
+      String user, String rowFilter) {
+    String json = String.format("{\n" +
+            "  \"name\": \"%s\",\n" +
+            "  \"policyType\": 2,\n" +
+            "  \"serviceType\": \"" + RANGER_SERVICE_TYPE + "\",\n" +
+            "  \"service\": \"" + RANGER_SERVICE_NAME + "\",\n" +
+            "  \"resources\": {\n" +
+            "    \"database\": {\n" +
+            "      \"values\": [\"%s\"],\n" +
+            "      \"isExcludes\": false,\n" +
+            "      \"isRecursive\": false\n" +
+            "    },\n" +
+            "    \"table\": {\n" +
+            "      \"values\": [\"%s\"],\n" +
+            "      \"isExcludes\": false,\n" +
+            "      \"isRecursive\": false\n" +
+            "    }\n" +
+            "  },\n" +
+            "  \"rowFilterPolicyItems\": [\n" +
+            "    {\n" +
+            "      \"accesses\": [\n" +
+            "        {\n" +
+            "          \"type\": \"select\",\n" +
+            "          \"isAllowed\": true\n" +
+            "        }\n" +
+            "      ],\n" +
+            "      \"users\": [\"%s\"],\n" +
+            "      \"rowFilterInfo\": {\"filterExpr\": \"%s\"}\n" +
+            "    }\n" +
+            "  ]\n" +
+            "}", policyName, dbName, tblName, user, rowFilter);
+    createRangerPolicy(policyName, json);
+  }
+
   /**
    * Test the error messages when Row Filtering is disabled.
    */
   @Test
-  public void testRowFilterEnabled() throws ImpalaException {
+  public void testRowFilterDisabled() throws ImpalaException {
     String policyName = "row_filter";
     for (String tableName: new String[]{"alltypes", "alltypes_view"}) {
       BackendConfig.INSTANCE.setRowFilteringEnabled(false);
-      String json = String.format("{\n" +
-          "  \"name\": \"%s\",\n" +
-          "  \"policyType\": 2,\n" +
-          "  \"serviceType\": \"%s\",\n" +
-          "  \"service\": \"%s\",\n" +
-          "  \"resources\": {\n" +
-          "    \"database\": {\n" +
-          "      \"values\": [\"functional\"],\n" +
-          "      \"isExcludes\": false,\n" +
-          "      \"isRecursive\": false\n" +
-          "    },\n" +
-          "    \"table\": {\n" +
-          "      \"values\": [\"%s\"],\n" +
-          "      \"isExcludes\": false,\n" +
-          "      \"isRecursive\": false\n" +
-          "    }\n" +
-          "  },\n" +
-          "  \"rowFilterPolicyItems\": [\n" +
-          "    {\n" +
-          "      \"accesses\": [\n" +
-          "        {\n" +
-          "          \"type\": \"select\",\n" +
-          "          \"isAllowed\": true\n" +
-          "        }\n" +
-          "      ],\n" +
-          "      \"users\": [\"%s\"],\n" +
-          "      \"rowFilterInfo\": {\"filterExpr\": \"id = 0\"}\n" +
-          "    }\n" +
-          "  ]\n" +
-          "}", policyName, RANGER_SERVICE_TYPE, RANGER_SERVICE_NAME, tableName,
-          user_.getShortName());
-
       try {
-        createRangerPolicy(policyName, json);
+        createRowFilteringPolicy(policyName, "functional", tableName,
+            user_.getShortName(), "id = 0");
         rangerImpalaPlugin_.refreshPoliciesAndTags();
 
         // Queries on tables that are not filtered should be allowed.
@@ -3061,6 +3073,145 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
   }
 
   /**
+   * Validates updates on data/metadata of table/view that has column-masking or
+   * row-filtering policies are blocked.
+   */
+  @Test
+  public void testUpdateOnMaskedTables() throws Exception {
+    try {
+      createRowFilteringPolicy("alltypes_row_filter", "functional", "alltypes",
+          user_.getShortName(), "id = 0");
+      createRowFilteringPolicy("alltypes_view_row_filter", "functional",
+          "alltypes_view", user_.getShortName(), "id = 0");
+      createColumnMaskingPolicy("alltypestiny_id_mask", "functional", "alltypestiny",
+          "id", user_.getShortName(), "CUSTOM", "id + 100");
+      createColumnMaskingPolicy("kudu_id_mask", "functional_kudu", "alltypes",
+          "id", user_.getShortName(), "MASK_NULL", /*maskExpr*/null);
+      // Add an unmasked policy. It should not block updates.
+      createColumnMaskingPolicy("alltypessmall_id_unmask", "functional", "alltypessmall",
+          "id", user_.getShortName(), "MASK_NONE", /*maskExpr*/null);
+      rangerImpalaPlugin_.refreshPoliciesAndTags();
+
+      // Select is ok.
+      authorize("select * from functional.alltypes")
+          .ok(onTable("functional", "alltypes", TPrivilegeLevel.SELECT));
+      authorize("select * from functional.alltypes_view")
+          .ok(onTable("functional", "alltypes_view", TPrivilegeLevel.SELECT));
+      authorize("select * from functional.alltypestiny")
+          .ok(onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
+
+      // Block INSERT, TRUNCATE even given SERVER ALL privilege
+      authorize("insert into functional.alltypes partition(year, month) " +
+          "select * from functional.alltypestiny")
+          .error(insertError("functional.alltypes"), onServer(TPrivilegeLevel.ALL))
+          // error for 'select' appears earlier than 'insert'
+          .error(selectError("functional.alltypestiny"),
+              onTable("functional", "alltypes", TPrivilegeLevel.ALL))
+          .error(selectError("functional.alltypestiny"),
+              onDatabase("functional", allExcept(
+                  TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.INSERT,
+                  TPrivilegeLevel.SELECT)));
+      authorize("truncate table functional.alltypes")
+          .error(insertError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("insert into functional.alltypestiny partition(year, month) " +
+          "select * from functional.alltypessmall")
+          .error(insertError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL))
+          // error for 'select' appears earlier than 'insert'
+          .error(selectError("functional.alltypessmall"),
+              onTable("functional", "alltypestiny", TPrivilegeLevel.ALL))
+          .error(selectError("functional.alltypessmall"),
+              onDatabase("functional", allExcept(
+                  TPrivilegeLevel.ALL, TPrivilegeLevel.OWNER, TPrivilegeLevel.INSERT,
+                  TPrivilegeLevel.SELECT)));
+      authorize("truncate table functional.alltypestiny")
+          .error(insertError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL));
+
+      // Block UPSERT, DELETE even given SERVER ALL privilege
+      authorize("upsert into functional_kudu.alltypes " +
+          "select * from functional.alltypes")
+          .error(accessError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL))
+          // error for 'select' appears earlier than 'access'
+          .error(selectError("functional.alltypes"),
+              onTable("functional_kudu", "alltypes", TPrivilegeLevel.ALL));
+      authorize("delete from functional_kudu.alltypes")
+          .error(accessError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("delete from functional_kudu.alltypes")
+          .error(accessError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("delete from functional_kudu.alltypes where id is not null")
+          .error(accessError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("delete a from functional_kudu.alltypes a join functional.alltypes b " +
+          "on a.id = b.id")
+          .error(accessError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL));
+
+      // Block compute stats even given SERVER ALL privilege
+      authorize("compute stats functional.alltypes")
+          .error(alterError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("compute incremental stats functional.alltypes")
+          .error(alterError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+
+      // Block ALTER even given SERVER ALL privilege
+      authorize("alter table functional.alltypes add columns (new_id int)")
+          .error(alterError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("alter table functional.alltypes drop partition (year=2009, month=1)")
+          .error(alterError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("alter view functional.alltypes_view as select 1")
+          .error(alterError("functional.alltypes_view"), onServer(TPrivilegeLevel.ALL));
+      authorize("alter table functional.alltypestiny add partition (year=1, month=1)")
+          .error(alterError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL));
+
+      // Block DROP even given SERVER ALL privilege
+      authorize("drop table functional.alltypes")
+          .error(dropError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("drop view functional.alltypes_view")
+          .error(dropError("functional.alltypes_view"), onServer(TPrivilegeLevel.ALL));
+      authorize("drop table functional.alltypestiny")
+          .error(dropError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL));
+      authorize("drop table functional_kudu.alltypes")
+          .error(dropError("functional_kudu.alltypes"), onServer(TPrivilegeLevel.ALL));
+
+      // Block REFRESH even given SERVER ALL privilege
+      authorize("refresh functional.alltypes")
+          .error(refreshError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("refresh functional.alltypes_view")
+          .error(refreshError("functional.alltypes_view"), onServer(TPrivilegeLevel.ALL));
+      authorize("refresh functional.alltypestiny")
+          .error(refreshError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL));
+      authorize("invalidate metadata functional.alltypes")
+          .error(refreshError("functional.alltypes"), onServer(TPrivilegeLevel.ALL));
+      authorize("invalidate metadata functional.alltypes_view")
+          .error(refreshError("functional.alltypes_view"), onServer(TPrivilegeLevel.ALL));
+      authorize("invalidate metadata functional.alltypestiny")
+          .error(refreshError("functional.alltypestiny"), onServer(TPrivilegeLevel.ALL));
+
+      // Unmasked policy won't block updates
+      authorize("insert into functional.alltypessmall partition(year, month) " +
+          "select * from functional.alltypestiny")
+          .ok(onTable("functional", "alltypessmall", TPrivilegeLevel.INSERT),
+              onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
+      authorize("alter table functional.alltypessmall add partition (year=1, month=1)")
+          .ok(onTable("functional", "alltypessmall", TPrivilegeLevel.ALTER));
+      authorize("compute stats functional.alltypessmall")
+          .ok(onTable("functional", "alltypessmall",
+              TPrivilegeLevel.ALTER, TPrivilegeLevel.SELECT));
+      authorize("compute incremental stats functional.alltypessmall")
+          .ok(onTable("functional", "alltypessmall",
+              TPrivilegeLevel.ALTER, TPrivilegeLevel.SELECT));
+      authorize("drop table functional.alltypessmall")
+          .ok(onTable("functional", "alltypessmall", TPrivilegeLevel.DROP));
+      authorize("refresh functional.alltypessmall")
+          .ok(onTable("functional", "alltypessmall", TPrivilegeLevel.REFRESH));
+      authorize("invalidate metadata functional.alltypessmall")
+          .ok(onTable("functional", "alltypessmall", TPrivilegeLevel.REFRESH));
+    } finally {
+      deleteRangerPolicy("alltypes_row_filter");
+      deleteRangerPolicy("alltypes_view_row_filter");
+      deleteRangerPolicy("alltypestiny_id_mask");
+      deleteRangerPolicy("kudu_id_mask");
+      deleteRangerPolicy("alltypessmall_id_unmask");
+    }
+  }
+
+  /**
    * Validates Ranger's object ownership privileges. Note that we no longer have to add a
    * policy to the Ranger server to explicitly grant a user the access privileges of the
    * resources if the user is the owner of the resources.
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 b2948bb..221dd41 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationTestBase.java
@@ -38,7 +38,6 @@ import org.apache.impala.thrift.TColumnValue;
 import org.apache.impala.thrift.TDescribeOutputStyle;
 import org.apache.impala.thrift.TDescribeResult;
 import org.apache.impala.thrift.TFunctionBinaryType;
-import org.apache.impala.thrift.TPrincipalType;
 import org.apache.impala.thrift.TPrivilege;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.impala.thrift.TPrivilegeScope;
@@ -47,15 +46,13 @@ import org.apache.impala.thrift.TTableName;
 import org.apache.ranger.plugin.util.GrantRevokeRequest;
 import org.apache.ranger.plugin.util.RangerRESTClient;
 import org.apache.ranger.plugin.util.RangerRESTUtils;
-import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.parser.JSONParser;
+import org.json.simple.parser.ParseException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status.Family;
-import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -549,7 +546,7 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
         "Principal: %s\nStatement: %s", withPrincipal.getName(), stmt));
   }
 
-  protected void createRangerPolicy(String policyName, String json) {
+  protected long createRangerPolicy(String policyName, String json) {
     ClientResponse response = rangerRestClient_
         .getResource("/service/public/v2/api/policy")
         .accept(RangerRESTUtils.REST_MIME_TYPE_JSON)
@@ -560,7 +557,17 @@ public abstract class AuthorizationTestBase extends FrontendTestBase {
           "Unable to create a Ranger policy: %s Response: %s",
           policyName, response.getEntity(String.class)));
     }
-    LOG.info("Created ranger policy {}: {}", policyName, json);
+    String content = response.getEntity(String.class);
+    JSONParser parser = new JSONParser();
+    long policyId = -1;
+    try {
+      Object obj = parser.parse(content);
+      policyId = (Long) ((JSONObject) obj).get("id");
+    } catch (ParseException e) {
+      LOG.error("Error parsing response content: {}", content);
+    }
+    LOG.info("Created ranger policy id={}, {}: {}", policyId, policyName, json);
+    return policyId;
   }
 
   protected void deleteRangerPolicy(String policyName) {
diff --git a/fe/src/test/java/org/apache/impala/authorization/ranger/RangerAuditLogTest.java b/fe/src/test/java/org/apache/impala/authorization/ranger/RangerAuditLogTest.java
index f0d0ce7..e2d8065 100644
--- a/fe/src/test/java/org/apache/impala/authorization/ranger/RangerAuditLogTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/ranger/RangerAuditLogTest.java
@@ -29,15 +29,17 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.thrift.TPrivilege;
 import org.apache.impala.thrift.TPrivilegeLevel;
 import org.apache.ranger.audit.model.AuthzAuditEvent;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 import java.util.function.Consumer;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
 public class RangerAuditLogTest extends AuthorizationTestBase {
@@ -284,6 +286,8 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
         "    \"dataMaskType\": \"MASK\"\n" +
         "  }\n"
     };
+    long policyIds[] = {-1, -1, -1, -1};
+    Set<Long> columnMaskingPolicyIds = new HashSet<>();
 
     List<String> policies = new ArrayList<>();
     for (int i = 0; i < masks.length; ++i) {
@@ -331,7 +335,10 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
       for (int i = 0; i < masks.length; ++i) {
         String policyName = policyNames[i];
         String json = policies.get(i);
-        createRangerPolicy(policyName, json);
+        policyIds[i] = createRangerPolicy(policyName, json);
+        assertNotEquals("Illegal policy id", -1, policyIds[i]);
+        // Only the first 3 policies apply on current user.
+        if (i < 3) columnMaskingPolicyIds.add(policyIds[i]);
       }
 
       authzOk(events -> {
@@ -348,6 +355,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(3));
         assertEventEquals("@column", "custom", "functional/alltypestiny/string_col", 1,
             events.get(4));
+        assertEquals(events.get(4).getPolicyId(), policyIds[0]);
       }, "select id, bool_col, string_col from functional.alltypestiny",
           onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
 
@@ -385,8 +393,10 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(13));
         assertEventEquals("@column", "mask_null",
             "functional/alltypestiny/date_string_col", 1, events.get(14));
+        assertEquals(events.get(14).getPolicyId(), policyIds[1]);
         assertEventEquals("@column", "custom", "functional/alltypestiny/string_col", 1,
             events.get(15));
+        assertEquals(events.get(15).getPolicyId(), policyIds[0]);
       }, "select * from functional.alltypestiny", onTable("functional", "alltypestiny",
           TPrivilegeLevel.SELECT));
 
@@ -411,6 +421,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(3));
         assertEventEquals("@column", "custom", "functional/alltypestiny/string_col", 1,
             events.get(4));
+        assertEquals(events.get(4).getPolicyId(), policyIds[0]);
       }, "with iv as (select id, bool_col, string_col from functional.alltypestiny) " +
           "select * from iv", onTable("functional", "alltypestiny",
           TPrivilegeLevel.SELECT));
@@ -429,6 +440,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(2));
         assertEventEquals("@column", "custom", "functional/alltypestiny/string_col", 1,
             events.get(3));
+        assertEquals(events.get(3).getPolicyId(), policyIds[0]);
       }, "select id, string_col from functional.alltypestiny a where exists " +
           "(select id from functional.alltypestiny where id = a.id) order by id;",
           onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
@@ -439,14 +451,73 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
       // event of the first column that failed the authorization will be logged, i.e.,
       // 'id'. Refer to RangerAuthorizationChecker#authorizeTableAccess() for further
       // details.
-      authzError(events -> {assertEquals(1, events.size());
-            assertEquals("with iv as (select id, bool_col, string_col from " +
-                    "functional.alltypestiny) select * from iv",
-                events.get(0).getRequestData());
-            assertEventEquals("@column", "select", "functional/alltypestiny/id", 0,
-                events.get(0));
-          },"with iv as (select id, bool_col, string_col from functional.alltypestiny) " +
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("with iv as (select id, bool_col, string_col from " +
+                "functional.alltypestiny) select * from iv",
+            events.get(0).getRequestData());
+        assertEventEquals("@column", "select", "functional/alltypestiny/id", 0,
+            events.get(0));
+      },"with iv as (select id, bool_col, string_col from functional.alltypestiny) " +
           "select * from iv", onTable("functional", "alltypestiny"));
+
+      // Updates on metadata fails by column-masking policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("invalidate metadata functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "refresh", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by a column masking policy.
+        assertTrue(columnMaskingPolicyIds.contains(events.get(0).getPolicyId()));
+      }, "invalidate metadata functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on metadata fails by column-masking policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("compute stats functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "alter", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by a column masking policy.
+        assertTrue(columnMaskingPolicyIds.contains(events.get(0).getPolicyId()));
+      }, "compute stats functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on metadata fails by column-masking policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("alter table functional.alltypestiny change column id id string",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "alter", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by a column masking policy.
+        assertTrue(columnMaskingPolicyIds.contains(events.get(0).getPolicyId()));
+      }, "alter table functional.alltypestiny change column id id string",
+          onServer(TPrivilegeLevel.ALL));
+
+      // Updates on data fails by column-masking policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("truncate table functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "insert", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by a column masking policy.
+        assertTrue(columnMaskingPolicyIds.contains(events.get(0).getPolicyId()));
+      }, "truncate table functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on data fails by column-masking policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("insert into functional.alltypestiny partition(year, month) " +
+                "select * from functional.alltypes",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "insert", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by a column masking policy.
+        assertTrue(columnMaskingPolicyIds.contains(events.get(0).getPolicyId()));
+      }, "insert into functional.alltypestiny partition(year, month) " +
+          "select * from functional.alltypes", onServer(TPrivilegeLevel.ALL));
     } finally {
       for (int i = 0; i < masks.length; ++i) {
         String policyName = policyNames[i];
@@ -465,6 +536,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
     String policyNames[] = {"tiny_filter", "all_filter"};
     String users[] = {user_.getShortName(), "non_owner_2"};
     String filters[] = {"id=0", "id=1"};
+    long policyIds[] = {-1, -1};
 
     List<String> policies = new ArrayList<>();
     for (int i = 0; i < filters.length; ++i) {
@@ -505,7 +577,8 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
       for (int i = 0; i < filters.length; ++i) {
         String policyName = policyNames[i];
         String json = policies.get(i);
-        createRangerPolicy(policyName, json);
+        policyIds[i] = createRangerPolicy(policyName, json);
+        assertNotEquals("Illegal policy id", -1, policyIds[i]);
       }
 
       // Verify row filter audits. Note that columns used in the row filter won't create
@@ -520,6 +593,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(1));
         assertEventEquals("@table", "row_filter", "functional/alltypestiny", 1,
             events.get(2));
+        assertEquals(events.get(2).getPolicyId(), policyIds[0]);
       }, "select bool_col from functional.alltypestiny",
           onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
 
@@ -531,6 +605,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(0));
         assertEventEquals("@table", "row_filter", "functional/alltypestiny", 1,
             events.get(1));
+        assertEquals(events.get(1).getPolicyId(), policyIds[0]);
       }, "select 1 from functional.alltypestiny",
           onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
 
@@ -542,6 +617,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(0));
         assertEventEquals("@table", "row_filter", "functional/alltypestiny", 1,
             events.get(1));
+        assertEquals(events.get(1).getPolicyId(), policyIds[0]);
       }, "select count(*) from functional.alltypestiny",
           onTable("functional", "alltypestiny", TPrivilegeLevel.SELECT));
 
@@ -589,6 +665,7 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
             events.get(13));
         assertEventEquals("@table", "row_filter", "functional/alltypestiny", 1,
             events.get(14));
+        assertEquals(events.get(14).getPolicyId(), policyIds[0]);
       }, "select * from functional.alltypestiny", onTable("functional", "alltypestiny",
           TPrivilegeLevel.SELECT));
 
@@ -631,16 +708,79 @@ public class RangerAuditLogTest extends AuthorizationTestBase {
       // When fails with not enough privileges, no audit logs for row filtering is
       // generated. Only the event of the first column (i.e. id) that failed the
       // authorization will be logged.
-      authzError(events -> {assertEquals(1, events.size());
+      authzError(events -> {
+        assertEquals(1, events.size());
         assertEquals("select * from functional.alltypestiny",
             events.get(0).getRequestData());
         assertEventEquals("@column", "select", "functional/alltypestiny/id", 0,
             events.get(0));
       },"select * from functional.alltypestiny", onTable("functional", "alltypestiny"));
+
+      // Updates on metadata fails by row-filtering policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("invalidate metadata functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "refresh", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by the row filtering policy.
+        assertEquals(events.get(0).getPolicyId(), policyIds[0]);
+      }, "invalidate metadata functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on metadata fails by row-filtering policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("compute stats functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "alter", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by the row filtering policy.
+        assertEquals(events.get(0).getPolicyId(), policyIds[0]);
+      }, "compute stats functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on metadata fails by row-filtering policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("alter table functional.alltypestiny change column id id string",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "alter", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by the row filtering policy.
+        assertEquals(events.get(0).getPolicyId(), policyIds[0]);
+      }, "alter table functional.alltypestiny change column id id string",
+          onServer(TPrivilegeLevel.ALL));
+
+      // Updates on data fails by row-filtering policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("truncate table functional.alltypestiny",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "insert", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by the row filtering policy.
+        assertEquals(events.get(0).getPolicyId(), policyIds[0]);
+      }, "truncate table functional.alltypestiny", onServer(TPrivilegeLevel.ALL));
+
+      // Updates on data fails by row-filtering policies.
+      authzError(events -> {
+        assertEquals(1, events.size());
+        assertEquals("insert into functional.alltypestiny partition(year, month) " +
+                "select * from functional.alltypes",
+            events.get(0).getRequestData());
+        assertEventEquals("@table", "insert", "functional/alltypestiny", 0,
+            events.get(0));
+        // Make sure it's denied by the row filtering policy.
+        assertEquals(events.get(0).getPolicyId(), policyIds[0]);
+      }, "insert into functional.alltypestiny partition(year, month) " +
+          "select * from functional.alltypes", onServer(TPrivilegeLevel.ALL));
     } finally {
       for (int i = 0; i < filters.length; ++i) {
         String policyName = policyNames[i];
-        deleteRangerPolicy(policyName);
+        try {
+          deleteRangerPolicy(policyName);
+        } catch (RuntimeException e) {
+          // ignore this to expose the original error.
+        }
       }
     }
   }
diff --git a/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking.test b/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking.test
index c0e37a4..c0b4a38 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking.test
@@ -414,3 +414,21 @@ select id, bigint_col from functional.alltypesagg order by id limit 10
 ---- TYPES
 INT,BIGINT
 ====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+insert into functional.alltypestiny partition(year, month) select * from functional.alltypes
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+truncate table functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+compute stats functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'ALTER' on: functional.alltypestiny
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking_and_row_filtering.test b/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking_and_row_filtering.test
index cc985ed..848ad4f 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking_and_row_filtering.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/ranger_column_masking_and_row_filtering.test
@@ -70,3 +70,21 @@ where int_col in
 ---- TYPES
 INT,INT
 ====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+insert into functional.alltypestiny partition(year, month) select * from functional.alltypes
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+truncate table functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+compute stats functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'ALTER' on: functional.alltypestiny
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/ranger_row_filtering.test b/testdata/workloads/functional-query/queries/QueryTest/ranger_row_filtering.test
index 2e5add8..c1ef6b9 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/ranger_row_filtering.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/ranger_row_filtering.test
@@ -499,3 +499,21 @@ NULL
 ---- TYPES
 INT
 ====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+insert into functional.alltypestiny partition(year, month) select * from functional.alltypes
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+truncate table functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'INSERT' on: functional.alltypestiny
+====
+---- QUERY
+# IMPALA-10554: Updates on masked tables should be blocked.
+compute stats functional.alltypestiny
+---- CATCH
+AuthorizationException: User '$USER' does not have privileges to execute 'ALTER' on: functional.alltypestiny
+====