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 @@ -345,7 +345,7 @@ object UnsupportedOperationChecker {
plan.foreachUp { implicit subPlan =>
subPlan match {
case (_: Project | _: Filter | _: MapElements | _: MapPartitions |
_: DeserializeToObject | _: SerializeFromObject) =>
_: DeserializeToObject | _: SerializeFromObject | _: SubqueryAlias) =>
case node if node.nodeName == "StreamingRelationV2" =>
case node =>
throwError(s"Continuous processing does not support ${node.nodeName} operations.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,25 @@ class ContinuousSuite extends ContinuousSuiteBase {
"Continuous processing does not support current time operations."))
}

test("subquery alias") {
val df = spark.readStream
.format("rate")
.option("numPartitions", "5")
.option("rowsPerSecond", "5")
.load()
.createOrReplaceTempView("rate")
val test = spark.sql("select value from rate where value > 5")

testStream(test, useV2Sink = true)(
StartStream(longContinuousTrigger),
AwaitEpoch(0),
Execute(waitForRateSourceTriggers(_, 2)),
IncrementEpoch(),
Execute(waitForRateSourceTriggers(_, 4)),
IncrementEpoch(),
CheckAnswerRowsContains(scala.Range(6, 20).map(Row(_))))
}

test("repeatedly restart") {
val df = spark.readStream
.format("rate")
Expand Down