-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24194] [SQL]HadoopFsRelation cannot overwrite a path that is also being read from. #21257
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,13 +17,14 @@ | |
|
|
||
| package org.apache.spark.internal.io | ||
|
|
||
| import java.io.IOException | ||
| import java.util.{Date, UUID} | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.util.Try | ||
|
|
||
| import org.apache.hadoop.conf.Configurable | ||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.fs.{FileSystem, Path} | ||
| import org.apache.hadoop.mapreduce._ | ||
| import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter | ||
| import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl | ||
|
|
@@ -84,6 +85,12 @@ class HadoopMapReduceCommitProtocol( | |
| */ | ||
| @transient private var partitionPaths: mutable.Set[String] = null | ||
|
|
||
| /** | ||
| * Tracks files will be delete when commit the job | ||
| */ | ||
| @transient private val pathsToDelete: mutable.Map[FileSystem, | ||
| mutable.Set[Path]] = mutable.HashMap() | ||
|
|
||
| /** | ||
| * The staging directory of this write job. Spark uses it to deal with files with absolute output | ||
| * path, or writing data into partitioned directory with dynamicPartitionOverwrite=true. | ||
|
|
@@ -163,6 +170,8 @@ class HadoopMapReduceCommitProtocol( | |
| } | ||
|
|
||
| override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { | ||
| cleanPathToDelete() | ||
|
|
||
| committer.commitJob(jobContext) | ||
|
|
||
| if (hasValidPath) { | ||
|
|
@@ -235,4 +244,41 @@ class HadoopMapReduceCommitProtocol( | |
| tmp.getFileSystem(taskContext.getConfiguration).delete(tmp, false) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * now just record the file to be delete | ||
| */ | ||
| override def deleteWithJob(fs: FileSystem, path: Path, | ||
| canDeleteNow: Boolean = true): Boolean = { | ||
| if (canDeleteNow) { | ||
| super.deleteWithJob(fs, path) | ||
| } else { | ||
| val set = if (pathsToDelete.contains(fs)) { | ||
| pathsToDelete(fs) | ||
| } else { | ||
| new mutable.HashSet[Path]() | ||
| } | ||
|
|
||
| set.add(path) | ||
| pathsToDelete.put(fs, set) | ||
| true | ||
| } | ||
| } | ||
|
|
||
| private def cleanPathToDelete(): Unit = { | ||
| // first delete the should delete special file | ||
| for (fs <- pathsToDelete.keys) { | ||
| for (path <- pathsToDelete(fs)) { | ||
| try { | ||
| if (!fs.delete(path, true)) { | ||
| logWarning(s"Delete path ${path} fail at job commit time") | ||
|
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. delete -> false just means there was nothing there, I wouldn't warn at that point. Unless |
||
| } | ||
| } catch { | ||
| case ex: IOException => | ||
| throw new IOException(s"Unable to clear output " + | ||
| s"file ${path} at job commit time", ex) | ||
|
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. recommend including ex.toString() in the new exception raised, as child exception text can often get lost |
||
| } | ||
| } | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -114,7 +114,9 @@ case class InsertIntoHadoopFsRelationCommand( | |
| // For dynamic partition overwrite, do not delete partition directories ahead. | ||
| true | ||
| } else { | ||
| deleteMatchingPartitions(fs, qualifiedOutputPath, customPartitionLocations, committer) | ||
| val outputCheck = DDLUtils.isInReadPath(query, outputPath) | ||
| deleteMatchingPartitions(fs, outputCheck, qualifiedOutputPath, | ||
| customPartitionLocations, committer) | ||
| true | ||
| } | ||
| case (SaveMode.Append, _) | (SaveMode.Overwrite, _) | (SaveMode.ErrorIfExists, false) => | ||
|
|
@@ -190,6 +192,7 @@ case class InsertIntoHadoopFsRelationCommand( | |
| */ | ||
| private def deleteMatchingPartitions( | ||
| fs: FileSystem, | ||
| outputCheck: Boolean, | ||
| qualifiedOutputPath: Path, | ||
| customPartitionLocations: Map[TablePartitionSpec, String], | ||
| committer: FileCommitProtocol): Unit = { | ||
|
|
@@ -207,9 +210,23 @@ case class InsertIntoHadoopFsRelationCommand( | |
| } | ||
| // first clear the path determined by the static partition keys (e.g. /table/foo=1) | ||
| val staticPrefixPath = qualifiedOutputPath.suffix(staticPartitionPrefix) | ||
| if (fs.exists(staticPrefixPath) && !committer.deleteWithJob(fs, staticPrefixPath, true)) { | ||
| throw new IOException(s"Unable to clear output " + | ||
| s"directory $staticPrefixPath prior to writing to it") | ||
| if (fs.exists(staticPrefixPath)) { | ||
| if (staticPartitionPrefix.isEmpty && outputCheck) { | ||
| // input contain output, only delete output sub files when job commit | ||
| val files = fs.listFiles(staticPrefixPath, false) | ||
|
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. if there are a lot of files here, you've gone from a dir delete which was O(1) on a fileystem, probably O(descendant) on an object store to at O(children) on an FS, O(children * descendants (chlld)) op here. Not significant for a small number of files, but could potentially be expensive. Why do the iteration at all? |
||
| while (files.hasNext) { | ||
| val file = files.next() | ||
| if (!committer.deleteWithJob(fs, file.getPath, false)) { | ||
| throw new IOException(s"Unable to clear output " + | ||
|
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. as |
||
| s"directory ${file.getPath} prior to writing to it") | ||
| } | ||
| } | ||
| } else { | ||
| if (!committer.deleteWithJob(fs, staticPrefixPath, true)) { | ||
| throw new IOException(s"Unable to clear output " + | ||
|
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. again, hard to see how this exception path would be reached. |
||
| s"directory $staticPrefixPath prior to writing to it") | ||
| } | ||
| } | ||
| } | ||
| // now clear all custom partition locations (e.g. /custom/dir/where/foo=2/bar=4) | ||
| for ((spec, customLoc) <- customPartitionLocations) { | ||
|
|
@@ -248,4 +265,5 @@ case class InsertIntoHadoopFsRelationCommand( | |
| } | ||
| }.toMap | ||
| } | ||
|
|
||
| } | ||
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.
No need to worry about concurrent access here, correct?