-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-28784][SS] Use CheckpointFileManager in StreamExecution/StreamingQueryManager for checkpoint dirs #25514
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
97bcf5b
9a92b5e
626992e
73ff29b
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 |
|---|---|---|
|
|
@@ -96,7 +96,8 @@ abstract class StreamExecution( | |
|
|
||
| val resolvedCheckpointRoot = { | ||
| val checkpointPath = new Path(checkpointRoot) | ||
| val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) | ||
| val checkpointFileManager = CheckpointFileManager.create(checkpointPath, | ||
| sparkSession.sessionState.newHadoopConf()) | ||
| if (sparkSession.conf.get(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED) | ||
| && StreamExecution.containsSpecialCharsInPath(checkpointPath)) { | ||
| // In Spark 2.4 and earlier, the checkpoint path is escaped 3 times (3 `Path.toUri.toString` | ||
|
|
@@ -106,7 +107,7 @@ abstract class StreamExecution( | |
| new Path(new Path(checkpointPath.toUri.toString).toUri.toString).toUri.toString | ||
| val legacyCheckpointDirExists = | ||
| try { | ||
| fs.exists(new Path(legacyCheckpointDir)) | ||
| checkpointFileManager.exists(new Path(legacyCheckpointDir)) | ||
| } catch { | ||
| case NonFatal(e) => | ||
| // We may not have access to this directory. Don't fail the query if that happens. | ||
|
|
@@ -133,9 +134,8 @@ abstract class StreamExecution( | |
| .stripMargin) | ||
| } | ||
| } | ||
| val checkpointDir = checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory) | ||
|
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.
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. I think so. We're using the
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. Since this ends-up in behavior change it may break existing queries. I think there must be a reason when we do such change. |
||
| fs.mkdirs(checkpointDir) | ||
| checkpointDir.toString | ||
| checkpointFileManager.mkdirs(checkpointPath) | ||
| checkpointPath.toString | ||
| } | ||
| logInfo(s"Checkpoint root $checkpointRoot resolved to $resolvedCheckpointRoot.") | ||
|
|
||
|
|
@@ -388,8 +388,9 @@ abstract class StreamExecution( | |
| val checkpointPath = new Path(resolvedCheckpointRoot) | ||
| try { | ||
| logInfo(s"Deleting checkpoint $checkpointPath.") | ||
| val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) | ||
| fs.delete(checkpointPath, true) | ||
| val manager = CheckpointFileManager.create(checkpointPath, | ||
| sparkSession.sessionState.newHadoopConf()) | ||
| manager.delete(checkpointPath) | ||
| } catch { | ||
| case NonFatal(e) => | ||
| // Deleting temp checkpoint folder is best effort, don't throw non fatal exceptions | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -444,7 +444,9 @@ class StreamSuite extends StreamTest { | |
| val inputData = MemoryStream[String] | ||
| val df = inputData.toDS().map(_ + "foo").groupBy("value").agg(count("*")) | ||
| // Test StreamingQuery.display | ||
| val dir1 = Utils.createTempDir().getCanonicalFile | ||
| val q = df.writeStream.queryName("memory_explain").outputMode("complete").format("memory") | ||
|
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. Why do we need this (and similar changes)? The original test is running fine with temp checkpoint. |
||
| .option("checkpointLocation", "file://" + dir1.getCanonicalPath) | ||
| .start() | ||
| .asInstanceOf[StreamingQueryWrapper] | ||
| .streamingQuery | ||
|
|
@@ -853,7 +855,6 @@ class StreamSuite extends StreamTest { | |
| test("should resolve the checkpoint path") { | ||
| withTempDir { dir => | ||
| val checkpointLocation = dir.getCanonicalPath | ||
| assert(!checkpointLocation.startsWith("file:/")) | ||
| val query = MemoryStream[Int].toDF | ||
| .writeStream | ||
| .option("checkpointLocation", checkpointLocation) | ||
|
|
@@ -862,7 +863,7 @@ class StreamSuite extends StreamTest { | |
| try { | ||
| val resolvedCheckpointDir = | ||
| query.asInstanceOf[StreamingQueryWrapper].streamingQuery.resolvedCheckpointRoot | ||
| assert(resolvedCheckpointDir.startsWith("file:/")) | ||
| assert(resolvedCheckpointDir.equals(checkpointLocation)) | ||
| } finally { | ||
| query.stop() | ||
| } | ||
|
|
||
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.
Here would it make sense to keep the fs check as well since we want to catch the previously created check point directories and suggest folks migrate?