@@ -20,9 +20,9 @@ package org.apache.spark.sql.execution.streaming
20
20
import org .apache .spark .rdd .RDD
21
21
import org .apache .spark .sql .SparkSession
22
22
import org .apache .spark .sql .catalyst .InternalRow
23
+ import org .apache .spark .sql .catalyst .analysis .MultiInstanceRelation
23
24
import org .apache .spark .sql .catalyst .expressions .Attribute
24
- import org .apache .spark .sql .catalyst .plans .logical .LeafNode
25
- import org .apache .spark .sql .catalyst .plans .logical .Statistics
25
+ import org .apache .spark .sql .catalyst .plans .logical .{LeafNode , LogicalPlan , Statistics }
26
26
import org .apache .spark .sql .execution .LeafExecNode
27
27
import org .apache .spark .sql .execution .datasources .DataSource
28
28
import org .apache .spark .sql .sources .v2 .{ContinuousReadSupport , DataSourceV2 }
@@ -42,7 +42,7 @@ object StreamingRelation {
42
42
* passing to [[StreamExecution ]] to run a query.
43
43
*/
44
44
case class StreamingRelation (dataSource : DataSource , sourceName : String , output : Seq [Attribute ])
45
- extends LeafNode {
45
+ extends LeafNode with MultiInstanceRelation {
46
46
override def isStreaming : Boolean = true
47
47
override def toString : String = sourceName
48
48
@@ -53,6 +53,8 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output:
53
53
override def computeStats (): Statistics = Statistics (
54
54
sizeInBytes = BigInt (dataSource.sparkSession.sessionState.conf.defaultSizeInBytes)
55
55
)
56
+
57
+ override def newInstance (): LogicalPlan = this .copy(output = output.map(_.newInstance()))
56
58
}
57
59
58
60
/**
@@ -62,7 +64,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output:
62
64
case class StreamingExecutionRelation (
63
65
source : BaseStreamingSource ,
64
66
output : Seq [Attribute ])(session : SparkSession )
65
- extends LeafNode {
67
+ extends LeafNode with MultiInstanceRelation {
66
68
67
69
override def isStreaming : Boolean = true
68
70
override def toString : String = source.toString
@@ -74,6 +76,8 @@ case class StreamingExecutionRelation(
74
76
override def computeStats (): Statistics = Statistics (
75
77
sizeInBytes = BigInt (session.sessionState.conf.defaultSizeInBytes)
76
78
)
79
+
80
+ override def newInstance (): LogicalPlan = this .copy(output = output.map(_.newInstance()))(session)
77
81
}
78
82
79
83
// 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(
92
96
extraOptions : Map [String , String ],
93
97
output : Seq [Attribute ],
94
98
v1Relation : Option [StreamingRelation ])(session : SparkSession )
95
- extends LeafNode {
99
+ extends LeafNode with MultiInstanceRelation {
96
100
override def isStreaming : Boolean = true
97
101
override def toString : String = sourceName
98
102
99
103
override def computeStats (): Statistics = Statistics (
100
104
sizeInBytes = BigInt (session.sessionState.conf.defaultSizeInBytes)
101
105
)
106
+
107
+ override def newInstance (): LogicalPlan = this .copy(output = output.map(_.newInstance()))(session)
102
108
}
103
109
104
110
/**
@@ -108,7 +114,7 @@ case class ContinuousExecutionRelation(
108
114
source : ContinuousReadSupport ,
109
115
extraOptions : Map [String , String ],
110
116
output : Seq [Attribute ])(session : SparkSession )
111
- extends LeafNode {
117
+ extends LeafNode with MultiInstanceRelation {
112
118
113
119
override def isStreaming : Boolean = true
114
120
override def toString : String = source.toString
@@ -120,6 +126,8 @@ case class ContinuousExecutionRelation(
120
126
override def computeStats (): Statistics = Statistics (
121
127
sizeInBytes = BigInt (session.sessionState.conf.defaultSizeInBytes)
122
128
)
129
+
130
+ override def newInstance (): LogicalPlan = this .copy(output = output.map(_.newInstance()))(session)
123
131
}
124
132
125
133
/**
0 commit comments