Skip to content

[SPARK-16430][SQL][STREAMING] Fixed bug in the maxFilesPerTrigger in FileStreamSource #14143

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

Closed
wants to merge 1 commit into from
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 @@ -73,8 +73,8 @@ class FileStreamSource(
logTrace(s"Number of seen files = ${seenFiles.size}")
if (batchFiles.nonEmpty) {
maxBatchId += 1
metadataLog.add(maxBatchId, newFiles)
logInfo(s"Max batch id increased to $maxBatchId with ${newFiles.size} new files")
metadataLog.add(maxBatchId, batchFiles)
logInfo(s"Max batch id increased to $maxBatchId with ${batchFiles.size} new files")
}

new LongOffset(maxBatchId)
Expand Down Expand Up @@ -138,7 +138,7 @@ class FileStreamSource(
.map { str =>
Try(str.toInt).toOption.filter(_ > 0).getOrElse {
throw new IllegalArgumentException(
s"Invalid value '$str' for option 'maxFilesPerBatch', must be a positive integer")
s"Invalid value '$str' for option 'maxFilesPerTrigger', must be a positive integer")
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -627,6 +627,13 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
checkAnswer(df, data.map(_.toString).toDF("value"))
}

def checkAllData(data: Seq[Int]): Unit = {
val schema = StructType(Seq(StructField("value", StringType)))
val df = spark.createDataFrame(
spark.sparkContext.makeRDD(memorySink.allData), schema)
checkAnswer(df, data.map(_.toString).toDF("value"))
}

/** Check how many batches have executed since the last time this check was made */
var lastBatchId = -1L
def checkNumBatchesSinceLastCheck(numBatches: Int): Unit = {
Expand All @@ -636,6 +643,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
}

checkLastBatchData(3) // (1 and 2) should be in batch 1, (3) should be in batch 2 (last)
checkAllData(1 to 3)
lastBatchId = memorySink.latestBatchId.get

fileSource.withBatchingLocked {
Expand All @@ -645,8 +653,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
createFile(7) // 6 and 7 should be in the last batch
}
q.processAllAvailable()
checkLastBatchData(6, 7)
checkNumBatchesSinceLastCheck(2)
checkLastBatchData(6, 7)
checkAllData(1 to 7)

fileSource.withBatchingLocked {
createFile(8)
Expand All @@ -656,8 +665,30 @@ class FileStreamSourceSuite extends FileStreamSourceTest {
createFile(12) // 12 should be in the last batch
}
q.processAllAvailable()
checkLastBatchData(12)
checkNumBatchesSinceLastCheck(3)
checkLastBatchData(12)
checkAllData(1 to 12)

q.stop()
}
}

test("max files per trigger - incorrect values") {
withTempDir { case src =>
def testMaxFilePerTriggerValue(value: String): Unit = {
val df = spark.readStream.option("maxFilesPerTrigger", value).text(src.getCanonicalPath)
val e = intercept[IllegalArgumentException] {
testStream(df)()
}
Seq("maxFilesPerTrigger", value, "positive integer").foreach { s =>
assert(e.getMessage.contains(s))
}
}

testMaxFilePerTriggerValue("not-a-integer")
testMaxFilePerTriggerValue("-1")
testMaxFilePerTriggerValue("0")
testMaxFilePerTriggerValue("10.1")
}
}

Expand Down