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 2022/04/06 12:56:31 UTC

[impala] branch master updated: IMPALA-10272: LOAD DATA should respect Ranger-HDFS policies

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 503e58174 IMPALA-10272: LOAD DATA should respect Ranger-HDFS policies
503e58174 is described below

commit 503e58174eaf5f0d6c052753ad0fabafefcc93f2
Author: LPL <li...@sensorsdata.cn>
AuthorDate: Thu Mar 31 19:36:24 2022 +0800

    IMPALA-10272: LOAD DATA should respect Ranger-HDFS policies
    
    When Ranger authorization is enabled, analyzing a LOAD DATA statement
    may fail when checking access to the source file if the HDFS permissions
    is more restrict than the Ranger policies.
    
    This patch provides a quick fix, when Ranger authz is enabled, we invoke
    method hadoop.fs.FileSystem.access(Path path, FsAction mode) to check
    the actual access permission first.
    
    Testing:
      - existing tests
      - add FsPermissionCheckerTest.java to fe ut
    
    Change-Id: Ia0acd83e8e237ea6a1d5fe1e9e440db0dd68b09c
    Reviewed-on: http://gerrit.cloudera.org:8080/18373
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 .../org/apache/impala/analysis/LoadDataStmt.java   | 13 +--
 .../apache/impala/util/FsPermissionChecker.java    | 21 +++++
 .../impala/util/FsPermissionCheckerTest.java       | 93 ++++++++++++++++++++++
 3 files changed, 121 insertions(+), 6 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/analysis/LoadDataStmt.java b/fe/src/main/java/org/apache/impala/analysis/LoadDataStmt.java
index 5aa7b4222..4ff13ec9e 100644
--- a/fe/src/main/java/org/apache/impala/analysis/LoadDataStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/LoadDataStmt.java
@@ -158,6 +158,7 @@ public class LoadDataStmt extends StatementBase {
       // TODO: Disable permission checking for S3A as well (HADOOP-13892)
       boolean shouldCheckPerms =
           FileSystemUtil.FsType.getFsType(fs.getScheme()) != FileSystemUtil.FsType.ADLS;
+      boolean authzEnabled = analyzer.isAuthzEnabled();
 
       if (fs.isDirectory(source)) {
         if (FileSystemUtil.getTotalNumVisibleFiles(source) == 0) {
@@ -169,8 +170,8 @@ public class LoadDataStmt extends StatementBase {
               "INPATH location '%s' cannot contain non-hidden subdirectories.",
               sourceDataPath_));
         }
-        if (!checker.getPermissions(fs, source).checkPermissions(
-            FsAction.READ_WRITE) && shouldCheckPerms) {
+        if (shouldCheckPerms && !checker
+          .checkAccess(source, fs, FsAction.READ_WRITE, authzEnabled)) {
           throw new AnalysisException(String.format("Unable to LOAD DATA from %s " +
               "because Impala does not have READ and WRITE permissions on this directory",
               source));
@@ -182,15 +183,15 @@ public class LoadDataStmt extends StatementBase {
               "INPATH location '%s' points to a hidden file.", source));
         }
 
-        if (!checker.getPermissions(fs, source.getParent()).checkPermissions(
-            FsAction.WRITE) && shouldCheckPerms) {
+        if (shouldCheckPerms && !checker
+          .checkAccess(source.getParent(), fs, FsAction.WRITE, authzEnabled)) {
           throw new AnalysisException(String.format("Unable to LOAD DATA from %s " +
               "because Impala does not have WRITE permissions on its parent " +
               "directory %s", source, source.getParent()));
         }
 
-        if (!checker.getPermissions(fs, source).checkPermissions(
-            FsAction.READ) && shouldCheckPerms) {
+        if (shouldCheckPerms && !checker
+          .checkAccess(source, fs, FsAction.READ, authzEnabled)) {
           throw new AnalysisException(String.format("Unable to LOAD DATA from %s " +
               "because Impala does not have READ permissions on this file", source));
         }
diff --git a/fe/src/main/java/org/apache/impala/util/FsPermissionChecker.java b/fe/src/main/java/org/apache/impala/util/FsPermissionChecker.java
index 497004341..17ca27eb6 100644
--- a/fe/src/main/java/org/apache/impala/util/FsPermissionChecker.java
+++ b/fe/src/main/java/org/apache/impala/util/FsPermissionChecker.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.hdfs.protocol.AclException;
 import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DeprecatedKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
@@ -310,6 +311,26 @@ public class FsPermissionChecker {
     return new Permissions(fileStatus, aclStatus);
   }
 
+  /**
+   * Returns true if the current user can perform the given action given these
+   * permissions.
+   */
+  public boolean checkAccess(Path path, FileSystem fs, FsAction action,
+      boolean isAuthzEnabled) throws IOException {
+    // When Ranger authz is enabled, we invoke method
+    // FileSystem#access(Path path, FsAction mode) to check the actual access permission.
+    if (isAuthzEnabled) {
+      try{
+        fs.access(path, action);
+      } catch (AccessControlException e) {
+        LOG.warn(e.getMessage());
+        return false;
+      }
+      return true;
+    }
+    return this.getPermissions(fs, path).checkPermissions(action);
+  }
+
   /**
    * Returns the FsPermissionChecker singleton.
    */
diff --git a/fe/src/test/java/org/apache/impala/util/FsPermissionCheckerTest.java b/fe/src/test/java/org/apache/impala/util/FsPermissionCheckerTest.java
new file mode 100644
index 000000000..fc4e3de4a
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/util/FsPermissionCheckerTest.java
@@ -0,0 +1,93 @@
+// 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.util;
+
+
+import java.io.IOException;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.security.AccessControlException;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+public class FsPermissionCheckerTest {
+
+  private final FsPermissionChecker checker = FsPermissionChecker.getInstance();
+
+  @Test
+  public void testCheckAccessWhenAuthorizationEnableAndFsPermissionDenied()
+      throws IOException {
+    Path path = new Path("hdfs://nameservice/user/impala/work");
+    FileSystem fs = Mockito.mock(FileSystem.class);
+    FsAction read = FsAction.READ;
+
+    Mockito.doThrow(new AccessControlException()).when(fs).access(path, read);
+
+    boolean access = checker.checkAccess(path, fs, read, true);
+    Assert.assertFalse(access);
+  }
+
+  @Test
+  public void testCheckAccessWhenAuthorizationEnableAndPermissionConfirmed()
+      throws IOException {
+    Path path = new Path("hdfs://nameservice/user/impala/work");
+    FileSystem fs = Mockito.mock(FileSystem.class);
+    FsAction read = FsAction.READ;
+
+    Mockito.doNothing().when(fs).access(path, read);
+
+    boolean access = checker.checkAccess(path, fs, read, true);
+    Assert.assertTrue(access);
+  }
+
+  @Test
+  public void testCheckAccessWhenAuthorizationDisableAndPermissionDenied()
+      throws IOException {
+    Path path = new Path("hdfs://nameservice/user/impala/work");
+    FileSystem fs = Mockito.mock(FileSystem.class);
+    FsAction read = FsAction.READ;
+
+    FileStatus fileStatus = Mockito.mock(FileStatus.class);
+    Mockito.doReturn(fileStatus).when(fs).getFileStatus(path);
+    FsPermission permission = FsPermission.valueOf("-r--r-----");
+    Mockito.doReturn(permission).when(fileStatus).getPermission();
+
+    boolean access = checker.checkAccess(path, fs, read, false);
+    Assert.assertFalse(access);
+  }
+
+  @Test
+  public void testCheckAccessWhenAuthorizationDisableAndPermissionConfirmed()
+      throws IOException {
+    Path path = new Path("hdfs://nameservice/user/impala/work");
+    FileSystem fs = Mockito.mock(FileSystem.class);
+    FsAction read = FsAction.READ;
+
+    FileStatus fileStatus = Mockito.mock(FileStatus.class);
+    Mockito.doReturn(fileStatus).when(fs).getFileStatus(path);
+    FsPermission permission = FsPermission.valueOf("-r--r--r--");
+    Mockito.doReturn(permission).when(fileStatus).getPermission();
+
+    boolean access = checker.checkAccess(path, fs, read, false);
+    Assert.assertTrue(access);
+  }
+}
\ No newline at end of file