Skip to content
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

[SPARK-23406] [SS] Enable stream-stream self-joins #20598

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -24,8 +24,8 @@ import scala.collection.mutable.{ArrayBuffer, Map => MutableMap}

import org.apache.spark.sql.{Dataset, SparkSession}
import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project}
import org.apache.spark.sql.execution.SQLExecution
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, WriteToDataSourceV2}
import org.apache.spark.sql.execution.streaming.sources.{InternalRowMicroBatchWriter, MicroBatchWriter}
Expand Down Expand Up @@ -431,7 +431,11 @@ class MicroBatchExecution(
s"Invalid batch: ${Utils.truncatedString(output, ",")} != " +
s"${Utils.truncatedString(dataPlan.output, ",")}")
replacements ++= output.zip(dataPlan.output)
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this is no longer used.

dataPlan

val aliases = output.zip(dataPlan.output).map { case (to, from) =>
Alias(from, to.name)(exprId = to.exprId, explicitMetadata = Some(from.metadata))
}
Project(aliases, dataPlan)
}.getOrElse {
LocalRelation(output, isStreaming = true)
}
Expand All @@ -440,8 +444,6 @@ class MicroBatchExecution(
// Rewire the plan to use the new attributes that were returned by the source.
val replacementMap = AttributeMap(replacements)
Copy link
Contributor

Choose a reason for hiding this comment

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

Same

val newAttributePlan = newBatchesPlan transformAllExpressions {
case a: Attribute if replacementMap.contains(a) =>
replacementMap(a).withMetadata(a.metadata)
case ct: CurrentTimestamp =>
CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs,
ct.dataType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.streaming
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LeafNode
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
import org.apache.spark.sql.execution.LeafExecNode
import org.apache.spark.sql.execution.datasources.DataSource
import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2}
Expand All @@ -42,7 +42,7 @@ object StreamingRelation {
* passing to [[StreamExecution]] to run a query.
*/
case class StreamingRelation(dataSource: DataSource, sourceName: String, output: Seq[Attribute])
extends LeafNode {
extends LeafNode with MultiInstanceRelation {
override def isStreaming: Boolean = true
override def toString: String = sourceName

Expand All @@ -53,6 +53,8 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output:
override def computeStats(): Statistics = Statistics(
sizeInBytes = BigInt(dataSource.sparkSession.sessionState.conf.defaultSizeInBytes)
)

override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))
}

/**
Expand All @@ -62,7 +64,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output:
case class StreamingExecutionRelation(
Copy link
Contributor

Choose a reason for hiding this comment

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

not very familiar with the streaming side, but IIRC, some of these plans are temporary and will be replaced before entering analyzer, and these plans don't need to extend MultiInstanceRelation.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

They need to extend MultiInstance relation, because Dataset.join() forces an analysis to disambiguate left and right in self-joins (here) and when there is a self-join between two streaming Datasets (i.e. they contain StreamingRelation/StreamingRelationV2), without the MultiInstanceRelation, it throws the error (see PR description).

Regarding StreamingExecutionRelation, while the other sources convert StreamingRelation to StreamingExecutionRelation, the MemoryStream directly injects StreamingExceutionRelation at that time of Dataset operations. Hence its good that StreamingExecutionRelation also extends MultiInstanceRelation.

source: BaseStreamingSource,
output: Seq[Attribute])(session: SparkSession)
extends LeafNode {
extends LeafNode with MultiInstanceRelation {

override def isStreaming: Boolean = true
override def toString: String = source.toString
Expand All @@ -74,6 +76,8 @@ case class StreamingExecutionRelation(
override def computeStats(): Statistics = Statistics(
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
)

override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))(session)
}

// We have to pack in the V1 data source as a shim, for the case when a source implements
Expand All @@ -92,13 +96,15 @@ case class StreamingRelationV2(
extraOptions: Map[String, String],
output: Seq[Attribute],
v1Relation: Option[StreamingRelation])(session: SparkSession)
extends LeafNode {
extends LeafNode with MultiInstanceRelation {
override def isStreaming: Boolean = true
override def toString: String = sourceName

override def computeStats(): Statistics = Statistics(
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
)

override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))(session)
}

/**
Expand All @@ -108,7 +114,7 @@ case class ContinuousExecutionRelation(
source: ContinuousReadSupport,
extraOptions: Map[String, String],
output: Seq[Attribute])(session: SparkSession)
extends LeafNode {
extends LeafNode with MultiInstanceRelation {

override def isStreaming: Boolean = true
override def toString: String = source.toString
Expand All @@ -120,6 +126,8 @@ case class ContinuousExecutionRelation(
override def computeStats(): Statistics = Statistics(
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
)

override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))(session)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,9 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession}
import org.apache.spark.sql.catalyst.analysis.StreamingJoinHelper
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Literal}
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, Filter}
import org.apache.spark.sql.execution.LogicalRDD
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.execution.{FileSourceScanExec, LogicalRDD}
import org.apache.spark.sql.execution.datasources.LogicalRelation
import org.apache.spark.sql.execution.streaming.{MemoryStream, StatefulOperatorStateInfo, StreamingSymmetricHashJoinHelper}
import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreProviderId}
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -323,6 +325,27 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with
assert(e.toString.contains("Stream stream joins without equality predicate is not supported"))
}

test("stream stream self join") {
val input = MemoryStream[Int]
val df = input.toDF
val join =
df.select('value % 5 as "key", 'value).join(
df.select('value % 5 as "key", 'value), "key")

testStream(join)(
AddData(input, 1, 2),
CheckAnswer((1, 1, 1), (2, 2, 2)),
StopStream,
StartStream(),
AddData(input, 3, 6),
/*
(1, 1) (1, 1)
(2, 2) x (2, 2) = (1, 1, 1), (1, 1, 6), (2, 2, 2), (1, 6, 1), (1, 6, 6)
(1, 6) (1, 6)
*/
CheckAnswer((3, 3, 3), (1, 1, 1), (1, 1, 6), (2, 2, 2), (1, 6, 1), (1, 6, 6)))
}

test("locality preferences of StateStoreAwareZippedRDD") {
import StreamingSymmetricHashJoinHelper._

Expand Down