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 @@ -18,6 +18,7 @@
package org.apache.spark.sql.execution.streaming

import scala.collection.mutable.{Map => MutableMap}
import scala.collection.mutable

import org.apache.spark.sql.{Dataset, SparkSession}
import org.apache.spark.sql.catalyst.encoders.RowEncoder
Expand Down Expand Up @@ -553,7 +554,7 @@ class MicroBatchExecution(
logDebug(s"Running batch $currentBatchId")

// Request unprocessed data from all sources.
newData = reportTimeTaken("getBatch") {
val mutableNewData = mutable.Map.empty ++ reportTimeTaken("getBatch") {
availableOffsets.flatMap {
case (source: Source, available: Offset)
if committedOffsets.get(source).map(_ != available).getOrElse(true) =>
Expand Down Expand Up @@ -590,7 +591,7 @@ class MicroBatchExecution(
val newBatchesPlan = logicalPlan transform {
// For v1 sources.
case StreamingExecutionRelation(source, output, catalogTable) =>
newData.get(source).map { dataPlan =>
mutableNewData.get(source).map { dataPlan =>
Copy link
Contributor

@HeartSaVioR HeartSaVioR Sep 16, 2022

Choose a reason for hiding this comment

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

While we are here, probably less intrusive change would be moving (L594 ~ L610) to L567. After the change we wouldn't need to make a change to newData here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks, I initially thought about that, but we need to know the output from StreamingExecutionRelation(source, output, catalogTable) to resolve _metadata right (L591 ~ L593)?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, you're right. I missed that.

Btw, looks like my change (tagging catalogTable into LogicalRelation) will also fall into this bug. Thanks for fixing this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Np - an unintentional fix :-)
Thanks for helping!

Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe, we may want to check the case of self-union / self-join to verify we really didn't break things. This works only when this condition is true leaf : source = 1 : 1 (otherwise we are overwriting the value in map), while the code comment of ProgressReporter tells there are counter cases.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Got it - could you share an example? In this case, does that mean the leaf : source = 1 : N?

Copy link
Contributor

Choose a reason for hiding this comment

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

The code comment actually doesn't say much and I'm speculating. Let's just try a best effort, self-union and self-join. df = spark.readStream... -> df.union(df) / df = spark.readStream... -> df.join(df)

val hasFileMetadata = output.exists {
Copy link
Contributor

Choose a reason for hiding this comment

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

looking at the code, seems the problem is we resolve the metadata columns in every micro-batch. Shouldn't we only resolve it once?

Copy link
Contributor

@HeartSaVioR HeartSaVioR Sep 16, 2022

Choose a reason for hiding this comment

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

It will require Source to indicate the request of metadata column and produce the logical plan accordingly when getBatch is called. My understanding is that DSv1 source does not have an interface to receive the information of which columns will be referred in actual query.

case FileSourceMetadataAttribute(_) => true
case _ => false
Expand All @@ -608,6 +609,11 @@ class MicroBatchExecution(
}
newRelation
}
// SPARK-40460: overwrite the entry with the new logicalPlan
// because it might contain the _metadata column. It is a necessary change,
// in the ProgressReporter, we use the following mapping to get correct streaming metrics:
// streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan
mutableNewData.put(source, finalDataPlan)
val maxFields = SQLConf.get.maxToStringFields
assert(output.size == finalDataPlan.output.size,
s"Invalid batch: ${truncatedString(output, ",", maxFields)} != " +
Expand All @@ -623,14 +629,14 @@ class MicroBatchExecution(

// For v2 sources.
case r: StreamingDataSourceV2Relation =>
newData.get(r.stream).map {
mutableNewData.get(r.stream).map {
case OffsetHolder(start, end) =>
r.copy(startOffset = Some(start), endOffset = Some(end))
}.getOrElse {
LocalRelation(r.output, isStreaming = true)
}
}

newData = mutableNewData.toMap
// Rewire the plan to use the new attributes that were returned by the source.
val newAttributePlan = newBatchesPlan.transformAllExpressionsWithPruning(
_.containsPattern(CURRENT_LIKE)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Ro
import org.apache.spark.sql.execution.FileSourceScanExec
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.Trigger
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructField, StructType}

Expand Down Expand Up @@ -518,16 +519,19 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession {
withTempDir { dir =>
df.coalesce(1).write.format("json").save(dir.getCanonicalPath + "/source/new-streaming-data")

val stream = spark.readStream.format("json")
val streamDf = spark.readStream.format("json")
.schema(schema)
.load(dir.getCanonicalPath + "/source/new-streaming-data")
.select("*", "_metadata")

val streamQuery0 = streamDf
.writeStream.format("json")
.option("checkpointLocation", dir.getCanonicalPath + "/target/checkpoint")
.trigger(Trigger.AvailableNow())
.start(dir.getCanonicalPath + "/target/new-streaming-data")

stream.processAllAvailable()
stream.stop()
streamQuery0.awaitTermination()
assert(streamQuery0.lastProgress.numInputRows == 2L)

val newDF = spark.read.format("json")
.load(dir.getCanonicalPath + "/target/new-streaming-data")
Expand Down Expand Up @@ -565,6 +569,34 @@ class FileMetadataStructSuite extends QueryTest with SharedSparkSession {
sourceFileMetadata(METADATA_FILE_MODIFICATION_TIME))
)
)

// Verify self-union
val streamQuery1 = streamDf.union(streamDf)
.writeStream.format("json")
.option("checkpointLocation", dir.getCanonicalPath + "/target/checkpoint_union")
.trigger(Trigger.AvailableNow())
.start(dir.getCanonicalPath + "/target/new-streaming-data-union")
streamQuery1.awaitTermination()
val df1 = spark.read.format("json")
.load(dir.getCanonicalPath + "/target/new-streaming-data-union")
// Verify self-union results
assert(streamQuery1.lastProgress.numInputRows == 4L)
assert(df1.count() == 4L)
assert(df1.select("*").columns.toSet == Set("name", "age", "info", "_metadata"))

// Verify self-join
val streamQuery2 = streamDf.join(streamDf, Seq("name", "age", "info", "_metadata"))
.writeStream.format("json")
.option("checkpointLocation", dir.getCanonicalPath + "/target/checkpoint_join")
.trigger(Trigger.AvailableNow())
.start(dir.getCanonicalPath + "/target/new-streaming-data-join")
streamQuery2.awaitTermination()
val df2 = spark.read.format("json")
.load(dir.getCanonicalPath + "/target/new-streaming-data-join")
// Verify self-join results
assert(streamQuery2.lastProgress.numInputRows == 4L)
assert(df2.count() == 2L)
assert(df2.select("*").columns.toSet == Set("name", "age", "info", "_metadata"))
}
}

Expand Down