You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2019/12/20 16:06:30 UTC

[GitHub] [spark] WinkerDu opened a new pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

WinkerDu opened a new pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971
 
 
   ### What changes were proposed in this pull request?
   Task attempt id is append to dynamic partition staging dir, commitTask in HadoopMapReduceCommitProtocol carries attempt id in TaskCommitMessage
   
   ### Why are the changes needed?
   This PR fix insert overwrite to DataSource table with dynamic partition error when running multiple task attempts. suppose there are one task attempt and one speculative task attempt, the speculative would raise FileAlreadyExistsException because of same staging dir attempt tasks commit
   
   ### Does this PR introduce any user-facing change?
   This PR add a configuration 'spark.max.local.task.failures' to set max failure time in LOCAL mode, default 1
   
   ### How was this patch tested?
   Added UT
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-568442052
 
 
   cc @cloud-fan 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r361285818
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 ##########
 @@ -167,8 +169,8 @@ class HadoopMapReduceCommitProtocol(
     committer.commitJob(jobContext)
 
     if (hasValidPath) {
-      val (allAbsPathFiles, allPartitionPaths) =
-        taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip
+      val (allAbsPathFiles, allPartitionPaths, successAttemptIDs) =
 
 Review comment:
   Should we add implicit function like 
   ```
   implicit def asPair(x:(Map[String, String], Set[String], String)) = (x._1, (x._2, x._3))
   ```
   before line 172, then we can `unzip` taskCommits as `(Map[String, String], (Set[String], String))` and eliminate re-`zip` operation at line 188
   
   Is it right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-567980140
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-567980140
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] jiangxb1987 commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
jiangxb1987 commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r379190648
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
 ##########
 @@ -1521,4 +1521,10 @@ package object config {
     .bytesConf(ByteUnit.BYTE)
     .createOptional
 
+  private[spark] val MAX_LOCAL_TASK_FAILURES = ConfigBuilder("spark.task.local.maxFailures")
+    .doc("The max failure times for a task while SparkContext running in Local mode, " +
 
 Review comment:
   How could you launch speculative task when running under local mode?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r361285818
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 ##########
 @@ -167,8 +169,8 @@ class HadoopMapReduceCommitProtocol(
     committer.commitJob(jobContext)
 
     if (hasValidPath) {
-      val (allAbsPathFiles, allPartitionPaths) =
-        taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip
+      val (allAbsPathFiles, allPartitionPaths, successAttemptIDs) =
 
 Review comment:
   Should we add new implicit function like '
   ```
   implicit def asPair(x:(Map[String, String], Set[String], String)) = (x._1, (x._2, x._3))
   ```
   before line 172, then we can `unzip` taskCommits as `(Map[String, String], (Set[String], String))` and reduce re-`zip` at line 188

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] WinkerDu commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
WinkerDu commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r379874760
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/config/package.scala
 ##########
 @@ -1521,4 +1521,10 @@ package object config {
     .bytesConf(ByteUnit.BYTE)
     .createOptional
 
+  private[spark] val MAX_LOCAL_TASK_FAILURES = ConfigBuilder("spark.task.local.maxFailures")
+    .doc("The max failure times for a task while SparkContext running in Local mode, " +
 
 Review comment:
   In UT class InsertWithMultipleTaskAttemptSuite, I don't expect launching speculative task in local mode. Actually, I make a customized commit protocol named "InsertExceptionCommitProtocol" in InsertWithMultipleTaskAttemptSuite, which overriding commitTask method to fail the first commit task on purpose and restore to normal in subsequent commit tasks. This scene is similar to what happened in speculative tasks with failure.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] WinkerDu commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
WinkerDu commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r361312369
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 ##########
 @@ -167,8 +169,8 @@ class HadoopMapReduceCommitProtocol(
     committer.commitJob(jobContext)
 
     if (hasValidPath) {
-      val (allAbsPathFiles, allPartitionPaths) =
-        taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip
+      val (allAbsPathFiles, allPartitionPaths, successAttemptIDs) =
 
 Review comment:
   Sounds reasonable, I'll try implicit conversion

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] koertkuipers commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
koertkuipers commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r404355749
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 ##########
 @@ -183,22 +187,30 @@ class HadoopMapReduceCommitProtocol(
       }
 
       if (dynamicPartitionOverwrite) {
-        val partitionPaths = allPartitionPaths.foldLeft(Set[String]())(_ ++ _)
-        logDebug(s"Clean up default partition directories for overwriting: $partitionPaths")
-        for (part <- partitionPaths) {
-          val finalPartPath = new Path(path, part)
-          if (!fs.delete(finalPartPath, true) && !fs.exists(finalPartPath.getParent)) {
-            // According to the official hadoop FileSystem API spec, delete op should assume
-            // the destination is no longer present regardless of return value, thus we do not
-            // need to double check if finalPartPath exists before rename.
-            // Also in our case, based on the spec, delete returns false only when finalPartPath
-            // does not exist. When this happens, we need to take action if parent of finalPartPath
-            // also does not exist(e.g. the scenario described on SPARK-23815), because
-            // FileSystem API spec on rename op says the rename dest(finalPartPath) must have
-            // a parent that exists, otherwise we may get unexpected result on the rename.
-            fs.mkdirs(finalPartPath.getParent)
-          }
-          fs.rename(new Path(stagingDir, part), finalPartPath)
+        val allPartitionPaths = partitionPathsAttemptIDPair.map {
+          case (allPartitionPath, successAttemptID) =>
+            allPartitionPath.foreach(part => {
+              val finalPartPath = new Path(path, part)
+              if (!fs.delete(finalPartPath, true) && !fs.exists(finalPartPath.getParent)) {
+                // According to the official hadoop FileSystem API spec, delete op should assume
+                // the destination is no longer present regardless of return value, thus we do not
+                // need to double check if finalPartPath exists before rename.
+                // Also in our case, based on the spec, delete returns false only when finalPartPath
+                // does not exist. When this happens, we need to take action if parent of
+                // finalPartPath also does not exist(e.g. the scenario described on SPARK-23815),
+                // because FileSystem API spec on rename op says the rename dest(finalPartPath)
+                // must have a parent that exists, otherwise we may get unexpected result
+                // on the rename.
+                fs.mkdirs(finalPartPath.getParent)
+              }
+              fs.rename(new Path(s"$stagingDir/$successAttemptID", part), finalPartPath)
 
 Review comment:
   do i understand it correctly that part here is a directory (e.g. x=1/y=2), not a file? so a directory full of files is being moved.
   if so couldn't multiple tasks write to the same partition? and then wouldnt these moves conflict with each other?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-595093491
 
 
   cc @dongjoon-hyun  please have a review?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-567991118
 
 
   Can one of the admins verify this patch?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
WinkerDu commented on issue #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#issuecomment-567984933
 
 
   @xuanyuanking @LuciferYang @LinhongLiu pls have a review

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on a change in pull request #26971: [SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error
URL: https://github.com/apache/spark/pull/26971#discussion_r361285818
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
 ##########
 @@ -167,8 +169,8 @@ class HadoopMapReduceCommitProtocol(
     committer.commitJob(jobContext)
 
     if (hasValidPath) {
-      val (allAbsPathFiles, allPartitionPaths) =
-        taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip
+      val (allAbsPathFiles, allPartitionPaths, successAttemptIDs) =
 
 Review comment:
   Should we add implicit function like 
   ```
   implicit def asPair(x:(Map[String, String], Set[String], String)) = (x._1, (x._2, x._3))
   ```
   before line 172, then we can `unzip` taskCommits as `(Map[String, String], (Set[String], String))` and reduce re-`zip` at line 188

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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