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

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

commit 43b6093dc00a746f8617dc6e1a63fef2dd82d76b
Author: Tim Armstrong <[email protected]>
AuthorDate: Tue Dec 22 11:42:42 2020 -0800

    IMPALA-10117: Skip calls to FsPermissionCache for blob stores
    
    This avoids calling precacheChildrenOf() in cases when the
    cached values will never be used. This change simply skips
    calling precacheChildrenOf() in the cases when getPermissions()
    is never called.
    
    There is some opportunity to clean up this permissions
    checking further, but I decided to keep this fix limited
    in scope.
    
    Change-Id: I2034695a956307309f656d56aa57aa07ae5163d8
    Reviewed-on: http://gerrit.cloudera.org:8080/16898
    Reviewed-by: Impala Public Jenkins <[email protected]>
    Tested-by: Impala Public Jenkins <[email protected]>
---
 .../java/org/apache/impala/catalog/HdfsTable.java  | 41 ++++++++++++++--------
 1 file changed, 27 insertions(+), 14 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java 
b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 05520e2..82f237c 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
 import javax.annotation.Nullable;
+
 import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -769,20 +770,8 @@ public class HdfsTable extends Table implements FeFsTable {
       Path location, FsPermissionCache permCache) throws IOException {
     Preconditions.checkNotNull(location);
     FileSystem fs = location.getFileSystem(CONF);
-    // Avoid calling getPermissions() on file path for S3 files, as that makes 
a round
-    // trip to S3. Also, the S3A connector is currently unable to manage S3 
permissions,
-    // so for now it is safe to assume that all files(objects) have READ_WRITE
-    // permissions, as that's what the S3A connector will always return too.
-    // TODO: Revisit if the S3A connector is updated to be able to manage S3 
object
-    // permissions. (see HADOOP-13892)
-    if (FileSystemUtil.isS3AFileSystem(fs)) return TAccessLevel.READ_WRITE;
 
-    // The ADLS connector currently returns ACLs for files in ADLS, but can 
only map
-    // them to the ADLS client SPI and not the Hadoop users/groups, causing 
unexpected
-    // behavior. So ADLS ACLs are unsupported until the connector is able to 
map
-    // permissions to hadoop users/groups (HADOOP-14437).
-    if (FileSystemUtil.isADLFileSystem(fs)) return TAccessLevel.READ_WRITE;
-    if (FileSystemUtil.isABFSFileSystem(fs)) return TAccessLevel.READ_WRITE;
+    if (assumeReadWriteAccess(fs)) return TAccessLevel.READ_WRITE;
 
     while (location != null) {
       try {
@@ -805,6 +794,28 @@ public class HdfsTable extends Table implements FeFsTable {
   }
 
   /**
+   * @return true if we assume read-write access for this filesystem for the 
purpose of
+   *              {@link #getAvailableAccessLevel(String, Path, 
FsPermissionCache)}
+   */
+  private static boolean assumeReadWriteAccess(FileSystem fs) {
+    // Avoid calling getPermissions() on file path for S3 files, as that makes 
a round
+    // trip to S3. Also, the S3A connector is currently unable to manage S3 
permissions,
+    // so for now it is safe to assume that all files(objects) have READ_WRITE
+    // permissions, as that's what the S3A connector will always return too.
+    // TODO: Revisit if the S3A connector is updated to be able to manage S3 
object
+    // permissions. (see HADOOP-13892)
+    if (FileSystemUtil.isS3AFileSystem(fs)) return true;
+
+    // The ADLS connector currently returns ACLs for files in ADLS, but can 
only map
+    // them to the ADLS client SPI and not the Hadoop users/groups, causing 
unexpected
+    // behavior. So ADLS ACLs are unsupported until the connector is able to 
map
+    // permissions to hadoop users/groups (HADOOP-14437).
+    if (FileSystemUtil.isADLFileSystem(fs)) return true;
+    if (FileSystemUtil.isABFSFileSystem(fs)) return true;
+    return false;
+  }
+
+  /**
    * Creates new HdfsPartition objects to be added to HdfsTable's partition 
list.
    * Partitions may be empty, or may not even exist in the filesystem (a 
partition's
    * location may have been changed to a new path that is about to be created 
by an
@@ -1817,7 +1828,9 @@ public class HdfsTable extends Table implements FeFsTable 
{
       Path p = entry.getElement();
       try {
         FileSystem fs = p.getFileSystem(CONF);
-        permCache.precacheChildrenOf(fs, p);
+        // Only attempt to cache permissions for filesystems where we will 
actually
+        // use them.
+        if (!assumeReadWriteAccess(fs)) permCache.precacheChildrenOf(fs, p);
       } catch (IOException ioe) {
         // If we fail to pre-warm the cache we'll just wait for later when we
         // try to actually load the individual permissions, at which point

Reply via email to