Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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 metadata object of a specified batch id.
* - Allow the user to query metadata objects in a range of batch ids.
* - Allow the user to remove obsolete metadata
*/
trait MetadataLog[T] {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,6 +290,13 @@ class StreamExecution(
assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)),
s"Concurrent update to the log. Multiple streaming jobs detected for $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 {
awaitBatchLock.lock()
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.filter(! _.endsWith(".crc")) // Workaround for SPARK-17475
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is the space between ! and _ intentionally added? I saw other similar code not having a space.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think @frreiss added this to be more obvious. I don't really have a preference here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Either way is fine with me.

assert(toTest.size == 1 && toTest.head == "2")
true
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This line ("true") shouldn't be here. It makes the Assert always pass, even when the condition on the previous line isn't satisfied.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It still fails. There was an assert there.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah, yes. The previous like (146) should be just toTest.size == 1 && toTest.head == "2", with no "assert".

}
)
}

/**
* A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`.
*
Expand Down