-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-30320][SQL] Fix insert overwrite to DataSource table with dynamic partition error #26971
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.io.IOException | |
| import java.util.{Date, UUID} | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.language.implicitConversions | ||
| import scala.util.Try | ||
|
|
||
| import org.apache.hadoop.conf.Configurable | ||
|
|
@@ -110,7 +111,9 @@ class HadoopMapReduceCommitProtocol( | |
| assert(dir.isDefined, | ||
| "The dataset to be written must be partitioned when dynamicPartitionOverwrite is true.") | ||
| partitionPaths += dir.get | ||
| this.stagingDir | ||
| // For speculative or re-executing task, we need to assign different | ||
| // dir for each task attempt | ||
| new Path(this.stagingDir, taskContext.getTaskAttemptID.toString) | ||
| // For FileOutputCommitter it has its own staging path called "work path". | ||
| case f: FileOutputCommitter => | ||
| new Path(Option(f.getWorkPath).map(_.toString).getOrElse(path)) | ||
|
|
@@ -165,10 +168,11 @@ class HadoopMapReduceCommitProtocol( | |
|
|
||
| override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { | ||
| committer.commitJob(jobContext) | ||
|
|
||
| if (hasValidPath) { | ||
| val (allAbsPathFiles, allPartitionPaths) = | ||
| taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String])]).unzip | ||
| implicit def asPair(x: (Map[String, String], Set[String], String)) | ||
| : (Map[String, String], (Set[String], String)) = (x._1, (x._2, x._3)) | ||
| val (allAbsPathFiles, partitionPathsAttemptIDPair) = | ||
| taskCommits.map(_.obj.asInstanceOf[(Map[String, String], Set[String], String)]).unzip | ||
| val fs = stagingDir.getFileSystem(jobContext.getConfiguration) | ||
|
|
||
| val filesToMove = allAbsPathFiles.foldLeft(Map[String, String]())(_ ++ _) | ||
|
|
@@ -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) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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. |
||
| }) | ||
| allPartitionPath | ||
| case _ => Set.empty | ||
| } | ||
| if (log.isDebugEnabled) { | ||
| val partitionPaths = allPartitionPaths.foldLeft(Set[String]())(_ ++ _) | ||
| logDebug(s"Clean up default partition directories for overwriting: $partitionPaths") | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -243,7 +255,7 @@ class HadoopMapReduceCommitProtocol( | |
| logTrace(s"Commit task ${attemptId}") | ||
| SparkHadoopMapRedUtil.commitTask( | ||
| committer, taskContext, attemptId.getJobID.getId, attemptId.getTaskID.getId) | ||
| new TaskCommitMessage(addedAbsPathFiles.toMap -> partitionPaths.toSet) | ||
| new TaskCommitMessage((addedAbsPathFiles.toMap, partitionPaths.toSet, attemptId.toString)) | ||
| } | ||
|
|
||
| /** | ||
|
|
||
78 changes: 78 additions & 0 deletions
78
...core/src/test/scala/org/apache/spark/sql/sources/InsertWithMultipleTaskAttemptSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,78 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.sql.sources | ||
|
|
||
| import java.util.concurrent.atomic.AtomicBoolean | ||
|
|
||
| import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.internal.config | ||
| import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage | ||
| import org.apache.spark.sql.{QueryTest, Row} | ||
| import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode | ||
| import org.apache.spark.sql.test.SharedSparkSession | ||
|
|
||
| class InsertWithMultipleTaskAttemptSuite extends QueryTest with SharedSparkSession { | ||
| override def sparkConf: SparkConf = { | ||
| val conf = super.sparkConf | ||
| conf.set(config.MAX_LOCAL_TASK_FAILURES, 3) | ||
| } | ||
|
|
||
| test("it is allowed to insert into a table for dynamic partition overwrite " + | ||
| "while speculation on") { | ||
| withSQLConf( | ||
| SQLConf.PARTITION_OVERWRITE_MODE.key -> PartitionOverwriteMode.DYNAMIC.toString, | ||
| SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[InsertExceptionCommitProtocol].getName) { | ||
| withTable("insertTable") { | ||
| sql( | ||
| """ | ||
| |CREATE TABLE insertTable(i int, part1 int, part2 int) USING PARQUET | ||
| |PARTITIONED BY (part1, part2) | ||
| """.stripMargin) | ||
|
|
||
| sql( | ||
| """ | ||
| |INSERT OVERWRITE TABLE insertTable Partition(part1=1, part2) | ||
| |SELECT 1,2 | ||
| """.stripMargin) | ||
| checkAnswer(spark.table("insertTable"), Row(1, 1, 2)) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| class InsertExceptionCommitProtocol( | ||
| jobId: String, | ||
| path: String, | ||
| dynamicPartitionOverwrite: Boolean = false) | ||
| extends SQLHadoopMapReduceCommitProtocol(jobId, path, dynamicPartitionOverwrite) { | ||
| override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = | ||
| if (InsertExceptionCommitProtocol.inited) { | ||
| throw new Exception("test") | ||
| } else { | ||
| super.commitTask(taskContext) | ||
| } | ||
| } | ||
|
|
||
| object InsertExceptionCommitProtocol { | ||
| private val initedFlag = new AtomicBoolean(false) | ||
| def inited: Boolean = initedFlag.compareAndSet(false, true) | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How could you launch speculative task when running under local mode?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
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.