Skip to content
Snippets Groups Projects
Commit be9d57fc authored by petermaxlee's avatar petermaxlee Committed by Reynold Xin
Browse files

[SPARK-17513][SQL] Make StreamExecution garbage-collect its metadata

## What changes were proposed in this pull request?
This PR modifies StreamExecution such that it discards metadata for batches that have already been fully processed. I used the purge method that was added as part of SPARK-17235.

This is based on work by frreiss in #15067, but fixed the test case along with some typos.

## How was this patch tested?
A new test case in StreamingQuerySuite. The test case would fail without the changes in this pull request.

Author: petermaxlee <petermaxlee@gmail.com>
Author: frreiss <frreiss@us.ibm.com>

Closes #15126 from petermaxlee/SPARK-17513.
parent 26145a5a
No related branches found
No related tags found
No related merge requests found
...@@ -24,6 +24,7 @@ package org.apache.spark.sql.execution.streaming ...@@ -24,6 +24,7 @@ package org.apache.spark.sql.execution.streaming
* - Allow the user to query the latest batch id. * - Allow the user to query the latest batch id.
* - Allow the user to query the metadata object of a specified batch id. * - Allow the user to query the metadata object of a specified batch id.
* - Allow the user to query metadata objects in a range of batch ids. * - Allow the user to query metadata objects in a range of batch ids.
* - Allow the user to remove obsolete metadata
*/ */
trait MetadataLog[T] { trait MetadataLog[T] {
......
...@@ -290,6 +290,13 @@ class StreamExecution( ...@@ -290,6 +290,13 @@ class StreamExecution(
assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)),
s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId")
logInfo(s"Committed offsets for batch $currentBatchId.") logInfo(s"Committed offsets for batch $currentBatchId.")
// Now that we have logged the new batch, no further processing will happen for
// the previous batch, and it is safe to discard the old metadata.
// Note that purge is exclusive, i.e. it purges everything before currentBatchId.
// NOTE: If StreamExecution implements pipeline parallelism (multiple batches in
// flight at the same time), this cleanup logic will need to change.
offsetLog.purge(currentBatchId)
} else { } else {
awaitBatchLock.lock() awaitBatchLock.lock()
try { try {
......
...@@ -125,6 +125,30 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ...@@ -125,6 +125,30 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter {
) )
} }
testQuietly("StreamExecution metadata garbage collection") {
val inputData = MemoryStream[Int]
val mapped = inputData.toDS().map(6 / _)
// Run 3 batches, and then assert that only 1 metadata file is left at the end
// since the first 2 should have been purged.
testStream(mapped)(
AddData(inputData, 1, 2),
CheckAnswer(6, 3),
AddData(inputData, 1, 2),
CheckAnswer(6, 3, 6, 3),
AddData(inputData, 4, 6),
CheckAnswer(6, 3, 6, 3, 1, 1),
AssertOnQuery("metadata log should contain only one file") { q =>
val metadataLogDir = new java.io.File(q.offsetLog.metadataPath.toString)
val logFileNames = metadataLogDir.listFiles().toSeq.map(_.getName())
val toTest = logFileNames // Workaround for SPARK-17475
assert(toTest.size == 1 && toTest.head == "2")
true
}
)
}
/** /**
* A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`.
* *
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment