You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by do...@apache.org on 2020/02/12 22:43:21 UTC

[spark] branch branch-2.4 updated: [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table

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

dongjoon pushed a commit to branch branch-2.4
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-2.4 by this push:
     new cf9f955  [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table
cf9f955 is described below

commit cf9f955119633b3044acff70708d0266f4c148bc
Author: Liang-Chi Hsieh <li...@uber.com>
AuthorDate: Wed Feb 12 14:27:18 2020 -0800

    [SPARK-30797][SQL] Set tradition user/group/other permission to ACL entries when setting up ACLs in truncate table
    
    ### What changes were proposed in this pull request?
    
    This is a follow-up to the PR #26956. In #26956, the patch proposed to preserve path permission when truncating table. When setting up original ACLs, we need to set user/group/other permission as ACL entries too, otherwise if the path doesn't have default user/group/other ACL entries, ACL API will complain an error `Invalid ACL: the user, group and other entries are required.`.
    
     In short this change makes sure:
    
    1. Permissions for user/group/other are always kept into ACLs to work with ACL API.
    2. Other custom ACLs are still kept after TRUNCATE TABLE (#26956 did this).
    
    ### Why are the changes needed?
    
    Without this fix, `TRUNCATE TABLE` will get an error when setting up ACLs if there is no default default user/group/other ACL entries.
    
    ### Does this PR introduce any user-facing change?
    
    No
    
    ### How was this patch tested?
    
    Update unit test.
    
    Manual test on dev Spark cluster.
    
    Set ACLs for a table path without default user/group/other ACL entries:
    ```
    hdfs dfs -setfacl --set 'user:liangchi:rwx,user::rwx,group::r--,other::r--' /user/hive/warehouse/test.db/test_truncate_table
    
    hdfs dfs -getfacl /user/hive/warehouse/test.db/test_truncate_table
    # file: /user/hive/warehouse/test.db/test_truncate_table
    # owner: liangchi
    # group: supergroup
    user::rwx
    user:liangchi:rwx
    group::r--
    mask::rwx
    other::r--
    ```
    Then run `sql("truncate table test.test_truncate_table")`, it works by normally truncating the table and preserve ACLs.
    
    Closes #27548 from viirya/fix-truncate-table-permission.
    
    Lead-authored-by: Liang-Chi Hsieh <li...@uber.com>
    Co-authored-by: Liang-Chi Hsieh <vi...@gmail.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
    (cherry picked from commit 5b76367a9d0aaca53ce96ab7e555a596567e8335)
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 .../spark/sql/execution/command/tables.scala       | 32 ++++++++++++++++++++--
 .../spark/sql/execution/command/DDLSuite.scala     | 21 +++++++++++++-
 2 files changed, 49 insertions(+), 4 deletions(-)

diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index 5323bf65..28dc4a4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -21,12 +21,13 @@ import java.io.File
 import java.net.{URI, URISyntaxException}
 import java.nio.file.FileSystems
 
+import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 import scala.util.Try
 import scala.util.control.NonFatal
 
 import org.apache.hadoop.fs.{FileContext, FsConstants, Path}
-import org.apache.hadoop.fs.permission.{AclEntry, FsPermission}
+import org.apache.hadoop.fs.permission.{AclEntry, AclEntryScope, AclEntryType, FsAction, FsPermission}
 
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -500,12 +501,27 @@ case class TruncateTableCommand(
               }
             }
             optAcls.foreach { acls =>
+              val aclEntries = acls.asScala.filter(_.getName != null).asJava
+
+              // If the path doesn't have default ACLs, `setAcl` API will throw an error
+              // as it expects user/group/other permissions must be in ACL entries.
+              // So we need to add tradition user/group/other permission
+              // in the form of ACL.
+              optPermission.map { permission =>
+                aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
+                  AclEntryType.USER, permission.getUserAction()))
+                aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
+                  AclEntryType.GROUP, permission.getGroupAction()))
+                aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
+                  AclEntryType.OTHER, permission.getOtherAction()))
+              }
+
               try {
-                fs.setAcl(path, acls)
+                fs.setAcl(path, aclEntries)
               } catch {
                 case NonFatal(e) =>
                   throw new SecurityException(
-                    s"Failed to set original ACL $acls back to " +
+                    s"Failed to set original ACL $aclEntries back to " +
                       s"the created path: $path. Exception: ${e.getMessage}")
               }
             }
@@ -536,6 +552,16 @@ case class TruncateTableCommand(
     }
     Seq.empty[Row]
   }
+
+  private def newAclEntry(
+      scope: AclEntryScope,
+      aclType: AclEntryType,
+      permission: FsAction): AclEntry = {
+    new AclEntry.Builder()
+      .setScope(scope)
+      .setType(aclType)
+      .setPermission(permission).build()
+  }
 }
 
 /**
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 790f6fc..4526578 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -1963,6 +1963,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           // Set ACL to table path.
           val customAcl = new java.util.ArrayList[AclEntry]()
           customAcl.add(new AclEntry.Builder()
+            .setName("test")
             .setType(AclEntryType.USER)
             .setScope(AclEntryScope.ACCESS)
             .setPermission(FsAction.READ).build())
@@ -1982,8 +1983,26 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
           if (ignore) {
             assert(aclEntries.size() == 0)
           } else {
-            assert(aclEntries.size() == 1)
+            assert(aclEntries.size() == 4)
             assert(aclEntries.get(0) == customAcl.get(0))
+
+            // Setting ACLs will also set user/group/other permissions
+            // as ACL entries.
+            val user = new AclEntry.Builder()
+              .setType(AclEntryType.USER)
+              .setScope(AclEntryScope.ACCESS)
+              .setPermission(FsAction.ALL).build()
+            val group = new AclEntry.Builder()
+              .setType(AclEntryType.GROUP)
+              .setScope(AclEntryScope.ACCESS)
+              .setPermission(FsAction.ALL).build()
+            val other = new AclEntry.Builder()
+              .setType(AclEntryType.OTHER)
+              .setScope(AclEntryScope.ACCESS)
+              .setPermission(FsAction.ALL).build()
+            assert(aclEntries.get(1) == user)
+            assert(aclEntries.get(2) == group)
+            assert(aclEntries.get(3) == other)
           }
         }
       }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org