-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-23288][SS] Fix output metrics with parquet sink #20745
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 2 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 |
|---|---|---|
|
|
@@ -21,6 +21,7 @@ import java.util.Locale | |
|
|
||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} | ||
| import org.apache.spark.sql.{AnalysisException, DataFrame} | ||
| import org.apache.spark.sql.execution.DataSourceScanExec | ||
| import org.apache.spark.sql.execution.datasources._ | ||
|
|
@@ -405,4 +406,52 @@ class FileStreamSinkSuite extends StreamTest { | |
| } | ||
| } | ||
| } | ||
|
|
||
| test("SPARK-23288 writing and checking output metrics") { | ||
| Seq("parquet", "orc", "text", "json").foreach { format => | ||
| val inputData = MemoryStream[String] | ||
| val df = inputData.toDF() | ||
|
|
||
| val outputDir = Utils.createTempDir(namePrefix = "stream.output").getCanonicalPath | ||
| val checkpointDir = Utils.createTempDir(namePrefix = "stream.checkpoint").getCanonicalPath | ||
|
|
||
| var query: StreamingQuery = null | ||
|
|
||
| var numTasks = 0 | ||
| var recordsWritten: Long = 0L | ||
| var bytesWritten: Long = 0L | ||
| try { | ||
| spark.sparkContext.addSparkListener(new SparkListener() { | ||
| override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { | ||
| val outputMetrics = taskEnd.taskMetrics.outputMetrics | ||
| recordsWritten += outputMetrics.recordsWritten | ||
| bytesWritten += outputMetrics.bytesWritten | ||
| numTasks += 1 | ||
| } | ||
| }) | ||
|
|
||
| query = | ||
| df.writeStream | ||
| .option("checkpointLocation", checkpointDir) | ||
| .format(format) | ||
| .start(outputDir) | ||
|
|
||
| inputData.addData("1", "2", "3") | ||
| inputData.addData("4", "5") | ||
|
|
||
| failAfter(streamingTimeout) { | ||
| query.processAllAvailable() | ||
| } | ||
|
|
||
|
Member
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. nit: it's better to add the below statement here to avoid flakiness.
Contributor
Author
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. Added. |
||
| assert(numTasks === 2) | ||
|
Member
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 would just check
Contributor
Author
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. Fixed. |
||
| assert(recordsWritten === 5) | ||
| // This is heavily file type/version specific but should be filled | ||
| assert(bytesWritten > 0) | ||
| } finally { | ||
| if (query != null) { | ||
| 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.
we should use
withTempDirto clean up the temp directory at the end