-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-14257][SQL]Allow multiple continuous queries to be started from the same DataFrame #12049
Changes from 4 commits
50c39b8
6196790
ac51850
9b5f007
2a71026
527f55f
48d760e
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 |
---|---|---|
|
@@ -19,16 +19,33 @@ package org.apache.spark.sql.execution.streaming | |
|
||
import org.apache.spark.sql.catalyst.expressions.Attribute | ||
import org.apache.spark.sql.catalyst.plans.logical.LeafNode | ||
import org.apache.spark.sql.execution.datasources.DataSource | ||
|
||
object StreamingRelation { | ||
def apply(source: Source): StreamingRelation = | ||
StreamingRelation(source, source.schema.toAttributes) | ||
def apply(dataSource: DataSource): StreamingRelation = { | ||
val source = dataSource.createSource() | ||
StreamingRelation(dataSource, source.schema.toAttributes) | ||
} | ||
} | ||
|
||
/** | ||
* Used to link a streaming [[DataSource]] into a | ||
* [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. | ||
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. Maybe include a description of how this gets turned into a 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. Done |
||
*/ | ||
case class StreamingRelation(dataSource: DataSource, output: Seq[Attribute]) extends LeafNode { | ||
override def toString: String = dataSource.createSource().toString | ||
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. This could be expensive right? I'm not sure that we want to do that in a 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.
Added a |
||
} | ||
|
||
/** | ||
* Used to link a streaming [[Source]] of data into a | ||
* [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. | ||
*/ | ||
case class StreamingRelation(source: Source, output: Seq[Attribute]) extends LeafNode { | ||
case class StreamingExecutionRelation(source: Source, output: Seq[Attribute]) extends LeafNode { | ||
override def toString: String = source.toString | ||
} | ||
|
||
object StreamingExecutionRelation { | ||
def apply(source: Source): StreamingExecutionRelation = { | ||
StreamingExecutionRelation(source, source.schema.toAttributes) | ||
} | ||
} |
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.
nit: i don't see anything named
_logicalPlan