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 @@ -66,6 +66,7 @@ case class StreamingExecutionRelation(
output: Seq[Attribute])(session: SparkSession)
extends LeafNode with MultiInstanceRelation {

override def otherCopyArgs: Seq[AnyRef] = session :: Nil
override def isStreaming: Boolean = true
override def toString: String = source.toString

Expand Down Expand Up @@ -97,6 +98,7 @@ case class StreamingRelationV2(
output: Seq[Attribute],
v1Relation: Option[StreamingRelation])(session: SparkSession)
extends LeafNode with MultiInstanceRelation {
override def otherCopyArgs: Seq[AnyRef] = session :: Nil
override def isStreaming: Boolean = true
override def toString: String = sourceName

Expand All @@ -116,6 +118,7 @@ case class ContinuousExecutionRelation(
output: Seq[Attribute])(session: SparkSession)
extends LeafNode with MultiInstanceRelation {

override def otherCopyArgs: Seq[AnyRef] = session :: Nil
override def isStreaming: Boolean = true
override def toString: String = source.toString

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -831,6 +831,21 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
CheckLastBatch(("A", 1)))
}

test("StreamingRelationV2/StreamingExecutionRelation/ContinuousExecutionRelation.toJSON " +
"should not fail") {
val df = spark.readStream.format("rate").load()
assert(df.logicalPlan.toJSON.contains("StreamingRelationV2"))

testStream(df)(
AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingExecutionRelation"))
)

testStream(df, useV2Sink = true)(
StartStream(trigger = Trigger.Continuous(100)),
AssertOnQuery(_.logicalPlan.toJSON.contains("ContinuousExecutionRelation"))
)
}

/** Create a streaming DF that only execute one batch in which it returns the given static DF */
private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = {
require(!triggerDF.isStreaming)
Expand Down