Skip to content

Commit 8298173

Browse files
viiryadongjoon-hyun
authored andcommitted
[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 <liangchi@uber.com> Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com> Signed-off-by: Dongjoon Hyun <dhyun@apple.com> (cherry picked from commit 5b76367) Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
1 parent a5bf41f commit 8298173

File tree

2 files changed

+49
-4
lines changed
  • sql/core/src

2 files changed

+49
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala

Lines changed: 29 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,13 @@ package org.apache.spark.sql.execution.command
1919

2020
import java.net.{URI, URISyntaxException}
2121

22+
import scala.collection.JavaConverters._
2223
import scala.collection.mutable.ArrayBuffer
2324
import scala.util.Try
2425
import scala.util.control.NonFatal
2526

2627
import org.apache.hadoop.fs.{FileContext, FsConstants, Path}
27-
import org.apache.hadoop.fs.permission.{AclEntry, FsPermission}
28+
import org.apache.hadoop.fs.permission.{AclEntry, AclEntryScope, AclEntryType, FsAction, FsPermission}
2829

2930
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
3031
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -538,12 +539,27 @@ case class TruncateTableCommand(
538539
}
539540
}
540541
optAcls.foreach { acls =>
542+
val aclEntries = acls.asScala.filter(_.getName != null).asJava
543+
544+
// If the path doesn't have default ACLs, `setAcl` API will throw an error
545+
// as it expects user/group/other permissions must be in ACL entries.
546+
// So we need to add tradition user/group/other permission
547+
// in the form of ACL.
548+
optPermission.map { permission =>
549+
aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
550+
AclEntryType.USER, permission.getUserAction()))
551+
aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
552+
AclEntryType.GROUP, permission.getGroupAction()))
553+
aclEntries.add(newAclEntry(AclEntryScope.ACCESS,
554+
AclEntryType.OTHER, permission.getOtherAction()))
555+
}
556+
541557
try {
542-
fs.setAcl(path, acls)
558+
fs.setAcl(path, aclEntries)
543559
} catch {
544560
case NonFatal(e) =>
545561
throw new SecurityException(
546-
s"Failed to set original ACL $acls back to " +
562+
s"Failed to set original ACL $aclEntries back to " +
547563
s"the created path: $path. Exception: ${e.getMessage}")
548564
}
549565
}
@@ -574,6 +590,16 @@ case class TruncateTableCommand(
574590
}
575591
Seq.empty[Row]
576592
}
593+
594+
private def newAclEntry(
595+
scope: AclEntryScope,
596+
aclType: AclEntryType,
597+
permission: FsAction): AclEntry = {
598+
new AclEntry.Builder()
599+
.setScope(scope)
600+
.setType(aclType)
601+
.setPermission(permission).build()
602+
}
577603
}
578604

579605
abstract class DescribeCommandBase extends RunnableCommand {

sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala

Lines changed: 20 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2042,6 +2042,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
20422042
// Set ACL to table path.
20432043
val customAcl = new java.util.ArrayList[AclEntry]()
20442044
customAcl.add(new AclEntry.Builder()
2045+
.setName("test")
20452046
.setType(AclEntryType.USER)
20462047
.setScope(AclEntryScope.ACCESS)
20472048
.setPermission(FsAction.READ).build())
@@ -2061,8 +2062,26 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils {
20612062
if (ignore) {
20622063
assert(aclEntries.size() == 0)
20632064
} else {
2064-
assert(aclEntries.size() == 1)
2065+
assert(aclEntries.size() == 4)
20652066
assert(aclEntries.get(0) == customAcl.get(0))
2067+
2068+
// Setting ACLs will also set user/group/other permissions
2069+
// as ACL entries.
2070+
val user = new AclEntry.Builder()
2071+
.setType(AclEntryType.USER)
2072+
.setScope(AclEntryScope.ACCESS)
2073+
.setPermission(FsAction.ALL).build()
2074+
val group = new AclEntry.Builder()
2075+
.setType(AclEntryType.GROUP)
2076+
.setScope(AclEntryScope.ACCESS)
2077+
.setPermission(FsAction.ALL).build()
2078+
val other = new AclEntry.Builder()
2079+
.setType(AclEntryType.OTHER)
2080+
.setScope(AclEntryScope.ACCESS)
2081+
.setPermission(FsAction.ALL).build()
2082+
assert(aclEntries.get(1) == user)
2083+
assert(aclEntries.get(2) == group)
2084+
assert(aclEntries.get(3) == other)
20662085
}
20672086
}
20682087
}

0 commit comments

Comments
 (0)