@@ -23,13 +23,9 @@ import io.grpc.stub.StreamObserver
23
23
24
24
import org .apache .spark .connect .proto
25
25
import org .apache .spark .internal .Logging
26
- import org .apache .spark .sql .Row
27
- import org .apache .spark .sql .catalyst .encoders .RowEncoder
28
- import org .apache .spark .sql .catalyst .plans .logical .LogicalPlan
29
- import org .apache .spark .sql .classic .{DataFrame , Dataset }
30
26
import org .apache .spark .sql .connect .common .{DataTypeProtoConverter , InvalidPlanInput , StorageLevelProtoConverter }
31
27
import org .apache .spark .sql .connect .planner .SparkConnectPlanner
32
- import org .apache .spark .sql .execution .{CodegenMode , CommandExecutionMode , CostMode , ExtendedMode , FormattedMode , SimpleMode }
28
+ import org .apache .spark .sql .execution .{CodegenMode , CostMode , ExtendedMode , FormattedMode , SimpleMode }
33
29
import org .apache .spark .sql .types .{DataType , StructType }
34
30
import org .apache .spark .util .ArrayImplicits ._
35
31
@@ -62,25 +58,20 @@ private[connect] class SparkConnectAnalyzeHandler(
62
58
val session = sessionHolder.session
63
59
val builder = proto.AnalyzePlanResponse .newBuilder()
64
60
65
- def transformRelation (rel : proto.Relation ) = planner.transformRelation(rel, cachePlan = true )
66
-
67
- def getDataFrameWithoutExecuting (rel : LogicalPlan ): DataFrame = {
68
- val qe = session.sessionState.executePlan(rel, CommandExecutionMode .SKIP )
69
- new Dataset [Row ](qe, () => RowEncoder .encoderFor(qe.analyzed.schema))
70
- }
71
-
72
61
request.getAnalyzeCase match {
73
62
case proto.AnalyzePlanRequest .AnalyzeCase .SCHEMA =>
74
- val rel = transformRelation(request.getSchema.getPlan.getRoot)
75
- val schema = getDataFrameWithoutExecuting(rel ).schema
63
+ val schema =
64
+ sessionHolder.createDataFrame(request.getSchema.getPlan.getRoot, planner ).schema
76
65
builder.setSchema(
77
66
proto.AnalyzePlanResponse .Schema
78
67
.newBuilder()
79
68
.setSchema(DataTypeProtoConverter .toConnectProtoType(schema))
80
69
.build())
81
70
case proto.AnalyzePlanRequest .AnalyzeCase .EXPLAIN =>
82
- val rel = transformRelation(request.getExplain.getPlan.getRoot)
83
- val queryExecution = getDataFrameWithoutExecuting(rel).queryExecution
71
+ val queryExecution =
72
+ sessionHolder
73
+ .createDataFrame(request.getExplain.getPlan.getRoot, planner)
74
+ .queryExecution
84
75
val explainString = request.getExplain.getExplainMode match {
85
76
case proto.AnalyzePlanRequest .Explain .ExplainMode .EXPLAIN_MODE_SIMPLE =>
86
77
queryExecution.explainString(SimpleMode )
@@ -101,8 +92,8 @@ private[connect] class SparkConnectAnalyzeHandler(
101
92
.build())
102
93
103
94
case proto.AnalyzePlanRequest .AnalyzeCase .TREE_STRING =>
104
- val rel = transformRelation(request.getTreeString.getPlan.getRoot)
105
- val schema = getDataFrameWithoutExecuting(rel ).schema
95
+ val schema =
96
+ sessionHolder.createDataFrame(request.getTreeString.getPlan.getRoot, planner ).schema
106
97
val treeString = if (request.getTreeString.hasLevel) {
107
98
schema.treeString(request.getTreeString.getLevel)
108
99
} else {
@@ -115,26 +106,28 @@ private[connect] class SparkConnectAnalyzeHandler(
115
106
.build())
116
107
117
108
case proto.AnalyzePlanRequest .AnalyzeCase .IS_LOCAL =>
118
- val rel = transformRelation(request.getIsLocal.getPlan.getRoot)
119
- val isLocal = getDataFrameWithoutExecuting(rel ).isLocal
109
+ val isLocal =
110
+ sessionHolder.createDataFrame(request.getIsLocal.getPlan.getRoot, planner ).isLocal
120
111
builder.setIsLocal(
121
112
proto.AnalyzePlanResponse .IsLocal
122
113
.newBuilder()
123
114
.setIsLocal(isLocal)
124
115
.build())
125
116
126
117
case proto.AnalyzePlanRequest .AnalyzeCase .IS_STREAMING =>
127
- val rel = transformRelation(request.getIsStreaming.getPlan.getRoot)
128
- val isStreaming = getDataFrameWithoutExecuting(rel).isStreaming
118
+ val isStreaming =
119
+ sessionHolder
120
+ .createDataFrame(request.getIsStreaming.getPlan.getRoot, planner)
121
+ .isStreaming
129
122
builder.setIsStreaming(
130
123
proto.AnalyzePlanResponse .IsStreaming
131
124
.newBuilder()
132
125
.setIsStreaming(isStreaming)
133
126
.build())
134
127
135
128
case proto.AnalyzePlanRequest .AnalyzeCase .INPUT_FILES =>
136
- val rel = transformRelation(request.getInputFiles.getPlan.getRoot)
137
- val inputFiles = getDataFrameWithoutExecuting(rel ).inputFiles
129
+ val inputFiles =
130
+ sessionHolder.createDataFrame(request.getInputFiles.getPlan.getRoot, planner ).inputFiles
138
131
builder.setInputFiles(
139
132
proto.AnalyzePlanResponse .InputFiles
140
133
.newBuilder()
@@ -157,27 +150,27 @@ private[connect] class SparkConnectAnalyzeHandler(
157
150
.build())
158
151
159
152
case proto.AnalyzePlanRequest .AnalyzeCase .SAME_SEMANTICS =>
160
- val targetRel = transformRelation(request.getSameSemantics.getTargetPlan.getRoot)
161
- val otherRel = transformRelation (request.getSameSemantics.getOtherPlan .getRoot)
162
- val target = getDataFrameWithoutExecuting(targetRel)
163
- val other = getDataFrameWithoutExecuting(otherRel )
153
+ val target =
154
+ sessionHolder.createDataFrame (request.getSameSemantics.getTargetPlan .getRoot, planner )
155
+ val other =
156
+ sessionHolder.createDataFrame(request.getSameSemantics.getOtherPlan.getRoot, planner )
164
157
builder.setSameSemantics(
165
158
proto.AnalyzePlanResponse .SameSemantics
166
159
.newBuilder()
167
160
.setResult(target.sameSemantics(other)))
168
161
169
162
case proto.AnalyzePlanRequest .AnalyzeCase .SEMANTIC_HASH =>
170
- val rel = transformRelation(request.getSemanticHash.getPlan.getRoot)
171
- val semanticHash = getDataFrameWithoutExecuting(rel )
163
+ val semanticHash = sessionHolder
164
+ .createDataFrame(request.getSemanticHash.getPlan.getRoot, planner )
172
165
.semanticHash()
173
166
builder.setSemanticHash(
174
167
proto.AnalyzePlanResponse .SemanticHash
175
168
.newBuilder()
176
169
.setResult(semanticHash))
177
170
178
171
case proto.AnalyzePlanRequest .AnalyzeCase .PERSIST =>
179
- val rel = transformRelation(request.getPersist.getRelation)
180
- val target = getDataFrameWithoutExecuting(rel )
172
+ val target = sessionHolder
173
+ .createDataFrame(request.getPersist.getRelation, planner )
181
174
if (request.getPersist.hasStorageLevel) {
182
175
target.persist(
183
176
StorageLevelProtoConverter .toStorageLevel(request.getPersist.getStorageLevel))
@@ -187,8 +180,8 @@ private[connect] class SparkConnectAnalyzeHandler(
187
180
builder.setPersist(proto.AnalyzePlanResponse .Persist .newBuilder().build())
188
181
189
182
case proto.AnalyzePlanRequest .AnalyzeCase .UNPERSIST =>
190
- val rel = transformRelation(request.getUnpersist.getRelation)
191
- val target = getDataFrameWithoutExecuting(rel )
183
+ val target = sessionHolder
184
+ .createDataFrame(request.getUnpersist.getRelation, planner )
192
185
if (request.getUnpersist.hasBlocking) {
193
186
target.unpersist(request.getUnpersist.getBlocking)
194
187
} else {
@@ -197,8 +190,8 @@ private[connect] class SparkConnectAnalyzeHandler(
197
190
builder.setUnpersist(proto.AnalyzePlanResponse .Unpersist .newBuilder().build())
198
191
199
192
case proto.AnalyzePlanRequest .AnalyzeCase .GET_STORAGE_LEVEL =>
200
- val rel = transformRelation(request.getGetStorageLevel.getRelation)
201
- val target = getDataFrameWithoutExecuting(rel )
193
+ val target = sessionHolder
194
+ .createDataFrame(request.getGetStorageLevel.getRelation, planner )
202
195
val storageLevel = target.storageLevel
203
196
builder.setGetStorageLevel(
204
197
proto.AnalyzePlanResponse .GetStorageLevel
0 commit comments