[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209957030
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -626,6 +626,14 @@ object SQLConf {
 .stringConf
 .createWithDefault("parquet")
 
+  val DATA_SOURCE_TABLE_INHERIT_PERMS = 
buildConf("spark.sql.datasource.table.inherit.perms")
+.internal()
+.doc("Set this to true if the table directories should be inheriting 
the permission " +
+  "of the warehouse or database directory " +
--- End diff --

yes, but that does a different thing IIUC, ie. that copies the permissions 
from the warehouse dir for the new tables created there. Here we are not doing 
the same, we are copying the permissions from the previous table dir to the new 
table dir after the overwrite.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread wangyum
Github user wangyum commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209954635
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -626,6 +626,14 @@ object SQLConf {
 .stringConf
 .createWithDefault("parquet")
 
+  val DATA_SOURCE_TABLE_INHERIT_PERMS = 
buildConf("spark.sql.datasource.table.inherit.perms")
+.internal()
+.doc("Set this to true if the table directories should be inheriting 
the permission " +
+  "of the warehouse or database directory " +
--- End diff --

Copy from `hive.warehouse.subdir.inherit.perms`: 
https://github.com/apache/hive/blob/rel/release-1.2.2/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java#L1729


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209940807
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -261,4 +273,67 @@ case class InsertIntoHadoopFsRelationCommand(
   }
 }.toMap
   }
+
+  private def isExtendedAclEnabled(hadoopConf: Configuration): Boolean =
+hadoopConf.getBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY, 
DFS_NAMENODE_ACLS_ENABLED_DEFAULT)
+
+  private def getFullFileStatus(
+  conf: SQLConf,
+  hadoopConf: Configuration,
+  fs: FileSystem,
+  file: Path): (String, FsPermission, AclStatus) = {
+if (conf.isDataSouceTableInheritPerms && fs.exists(file)) {
+  val fileStatus = fs.getFileStatus(file)
+  val aclStatus = if (isExtendedAclEnabled(hadoopConf)) 
fs.getAclStatus(file) else null
+  (fileStatus.getGroup, fileStatus.getPermission, aclStatus)
+} else {
+  (null, null, null)
+}
+  }
+
+  private def setFullFileStatus(
+  hadoopConf: Configuration,
+  group: String,
+  permission: FsPermission,
+  aclStatus: AclStatus,
+  fs: FileSystem,
+  target: Path): Unit = {
+try {
+  // use FsShell to change group, permissions, and extended ACL's 
recursively
+  val fsShell = new FsShell
+  fsShell.setConf(hadoopConf)
+  fsShell.run(Array[String]("-chgrp", "-R", group, target.toString))
--- End diff --

Well, for things like setting the group/user you can use `fs.setOwner` for 
instance


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread wangyum
Github user wangyum commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209934062
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -261,4 +273,67 @@ case class InsertIntoHadoopFsRelationCommand(
   }
 }.toMap
   }
+
+  private def isExtendedAclEnabled(hadoopConf: Configuration): Boolean =
+hadoopConf.getBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY, 
DFS_NAMENODE_ACLS_ENABLED_DEFAULT)
+
+  private def getFullFileStatus(
+  conf: SQLConf,
+  hadoopConf: Configuration,
+  fs: FileSystem,
+  file: Path): (String, FsPermission, AclStatus) = {
+if (conf.isDataSouceTableInheritPerms && fs.exists(file)) {
+  val fileStatus = fs.getFileStatus(file)
+  val aclStatus = if (isExtendedAclEnabled(hadoopConf)) 
fs.getAclStatus(file) else null
+  (fileStatus.getGroup, fileStatus.getPermission, aclStatus)
+} else {
+  (null, null, null)
+}
+  }
+
+  private def setFullFileStatus(
+  hadoopConf: Configuration,
+  group: String,
+  permission: FsPermission,
+  aclStatus: AclStatus,
+  fs: FileSystem,
+  target: Path): Unit = {
+try {
+  // use FsShell to change group, permissions, and extended ACL's 
recursively
+  val fsShell = new FsShell
+  fsShell.setConf(hadoopConf)
+  fsShell.run(Array[String]("-chgrp", "-R", group, target.toString))
--- End diff --

These code copy from 
https://github.com/apache/hive/blob/rel/release-1.2.2/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java#L722-L767

I will think about whether there is a better implementation.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209859205
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala
 ---
@@ -261,4 +273,67 @@ case class InsertIntoHadoopFsRelationCommand(
   }
 }.toMap
   }
+
+  private def isExtendedAclEnabled(hadoopConf: Configuration): Boolean =
+hadoopConf.getBoolean(DFS_NAMENODE_ACLS_ENABLED_KEY, 
DFS_NAMENODE_ACLS_ENABLED_DEFAULT)
+
+  private def getFullFileStatus(
+  conf: SQLConf,
+  hadoopConf: Configuration,
+  fs: FileSystem,
+  file: Path): (String, FsPermission, AclStatus) = {
+if (conf.isDataSouceTableInheritPerms && fs.exists(file)) {
+  val fileStatus = fs.getFileStatus(file)
+  val aclStatus = if (isExtendedAclEnabled(hadoopConf)) 
fs.getAclStatus(file) else null
+  (fileStatus.getGroup, fileStatus.getPermission, aclStatus)
+} else {
+  (null, null, null)
+}
+  }
+
+  private def setFullFileStatus(
+  hadoopConf: Configuration,
+  group: String,
+  permission: FsPermission,
+  aclStatus: AclStatus,
+  fs: FileSystem,
+  target: Path): Unit = {
+try {
+  // use FsShell to change group, permissions, and extended ACL's 
recursively
+  val fsShell = new FsShell
+  fsShell.setConf(hadoopConf)
+  fsShell.run(Array[String]("-chgrp", "-R", group, target.toString))
--- End diff --

can we avoid using the shell ad rather use the programmatic API? eg. 
`fs.setOwner`.


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-14 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209859644
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -626,6 +626,14 @@ object SQLConf {
 .stringConf
 .createWithDefault("parquet")
 
+  val DATA_SOURCE_TABLE_INHERIT_PERMS = 
buildConf("spark.sql.datasource.table.inherit.perms")
+.internal()
+.doc("Set this to true if the table directories should be inheriting 
the permission " +
+  "of the warehouse or database directory " +
--- End diff --

I think this is not very clear: it seems to me that the permissions are 
inherited from the base directory where the table is going to be written. On 
the other hand, instead, the permissions are inherited from the previous table 
directory which was deleted, right? Can we make this more clear?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-11 Thread mgaido91
Github user mgaido91 commented on a diff in the pull request:

https://github.com/apache/spark/pull/22078#discussion_r209430770
  
--- Diff: 
core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala ---
@@ -124,6 +124,25 @@ abstract class FileCommitProtocol {
 fs.delete(path, recursive)
   }
 
+  /**
+   * Specifies that a directory should be truncated with the commit of 
this job. The default
+   * implementation deletes the file immediately.
+   *
+   * Return true if this directory is empty or deleted all files in this 
directory, otherwise false.
+   */
+  def truncateDirectoryWithJob(fs: FileSystem, directory: Path, recursive: 
Boolean): Boolean = {
+assert(fs.isDirectory(directory))
+val listStatus = fs.listStatus(directory)
+if (listStatus.isEmpty) {
+  true
+} else {
+  val result = listStatus.map { f =>
--- End diff --

what about `forall`?


---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #22078: [SPARK-25085][SQL] Insert overwrite a non-partiti...

2018-08-11 Thread wangyum
GitHub user wangyum opened a pull request:

https://github.com/apache/spark/pull/22078

[SPARK-25085][SQL] Insert overwrite a non-partitioned table should not 
delete table folder

## What changes were proposed in this pull request?

Insert overwrite a `non-partitioned` table should not delete table folder 
because it may contains information like ACL entries.
This pr fix this issue, thus the data source table and the hive table have 
the same behavior.
## How was this patch tested?

unit tests


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/wangyum/spark SPARK-25085

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/22078.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #22078


commit 36a545ce96c8d43c59584f82c3e13854bd2a21ed
Author: Yuming Wang 
Date:   2018-08-11T15:39:35Z

Fix SPARK-25085




---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org