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 @@ -540,7 +541,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 @@ -577,7 +578,7 @@ class MicroBatchExecution(
val newBatchesPlan = logicalPlan transform {
// For v1 sources.
case StreamingExecutionRelation(source, output) =>
newData.get(source).map { dataPlan =>
mutableNewData.get(source).map { dataPlan =>
val hasFileMetadata = output.exists {
case FileSourceMetadataAttribute(_) => true
case _ => false
Expand All @@ -586,6 +587,11 @@ class MicroBatchExecution(
case l: LogicalRelation if hasFileMetadata => l.withMetadataColumns()
case _ => dataPlan
}
// 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 @@ -601,14 +607,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