-
Notifications
You must be signed in to change notification settings - Fork 28.2k
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
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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} | ||
|
@@ -431,7 +431,11 @@ class MicroBatchExecution( | |
s"Invalid batch: ${Utils.truncatedString(output, ",")} != " + | ||
s"${Utils.truncatedString(dataPlan.output, ",")}") | ||
replacements ++= output.zip(dataPlan.output) | ||
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) | ||
} | ||
|
@@ -440,8 +444,6 @@ class MicroBatchExecution( | |
// Rewire the plan to use the new attributes that were returned by the source. | ||
val replacementMap = AttributeMap(replacements) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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} | ||
|
@@ -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 | ||
|
||
|
@@ -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())) | ||
} | ||
|
||
/** | ||
|
@@ -62,7 +64,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: | |
case class StreamingExecutionRelation( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 | ||
|
@@ -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 | ||
|
@@ -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) | ||
} | ||
|
||
/** | ||
|
@@ -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 | ||
|
@@ -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) | ||
} | ||
|
||
/** | ||
|
There was a problem hiding this comment.
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.