-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-34526][SS] Ignore the error when checking the path in FileStreamSink.hasMetadata #31638
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
83dd27a
36df5ee
7236644
54a3283
fa42b0f
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,17 +17,18 @@ | |
|
|
||
| package org.apache.spark.sql.streaming | ||
|
|
||
| import java.io.File | ||
| import java.io.{File, IOException} | ||
| import java.nio.file.Files | ||
| import java.util.Locale | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem} | ||
| import org.apache.hadoop.mapreduce.JobContext | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.deploy.SparkHadoopUtil | ||
| import org.apache.spark.internal.io.FileCommitProtocol | ||
| import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} | ||
| import org.apache.spark.sql.{AnalysisException, DataFrame} | ||
|
|
@@ -575,6 +576,30 @@ abstract class FileStreamSinkSuite extends StreamTest { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("formatCheck fail should not fail the query") { | ||
| withSQLConf( | ||
| "fs.file.impl" -> classOf[FailFormatCheckFileSystem].getName, | ||
|
||
| "fs.file.impl.disable.cache" -> "true") { | ||
| withTempDir { tempDir => | ||
| val path = new File(tempDir, "text").getCanonicalPath | ||
| Seq("foo").toDF.write.format("text").save(path) | ||
| spark.read.format("text").load(path) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| test("fail to check glob path should not fail the query") { | ||
| withSQLConf( | ||
| "fs.file.impl" -> classOf[FailFormatCheckFileSystem].getName, | ||
| "fs.file.impl.disable.cache" -> "true") { | ||
| withTempDir { tempDir => | ||
| val path = new File(tempDir, "text").getCanonicalPath | ||
| Seq("foo").toDF.write.format("text").save(path) | ||
| spark.read.format("text").load(path + "/*") | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
| object PendingCommitFilesTrackingManifestFileCommitProtocol { | ||
|
|
@@ -685,3 +710,19 @@ class FileStreamSinkV2Suite extends FileStreamSinkSuite { | |
| // TODO: test partition pruning when file source V2 supports it. | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * A special file system that fails when accessing metadata log directory or using a glob path to | ||
| * access. | ||
| */ | ||
| class FailFormatCheckFileSystem extends RawLocalFileSystem { | ||
| override def getFileStatus(f: Path): FileStatus = { | ||
| if (f.getName == FileStreamSink.metadataDir) { | ||
| throw new IOException("cannot access metadata log") | ||
| } | ||
| if (SparkHadoopUtil.get.isGlobPath(f)) { | ||
| throw new IOException("fail to access a glob path") | ||
| } | ||
| super.getFileStatus(f) | ||
| } | ||
| } | ||
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.
This looks to be different from 2.4:
https://github.com/apache/spark/blob/branch-2.4/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala
Do you have specific exceptions to catch here? I can't imagine any SparkException from above logic.
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.
This is added for fixing the UT
org.apache.spark.sql.streaming.StreamingQuerySuite.detect escaped path and report the migration guide, which expects to have a SparkException thrown incheckEscapedMetadataPath.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.
Ah OK I've missed that
checkEscapedMetadataPaththrows SparkException. My bad.