@@ -59,12 +59,13 @@ private[connect] class SparkConnectAnalyzeHandler(
59
59
val session = sessionHolder.session
60
60
val builder = proto.AnalyzePlanResponse .newBuilder()
61
61
62
- def transformRelation (rel : proto.Relation ) = planner.transformRelation(rel, cachePlan = true )
62
+ def transformRelation (rel : proto.Relation ) = planner.transformRelation(rel)
63
63
64
64
request.getAnalyzeCase match {
65
65
case proto.AnalyzePlanRequest .AnalyzeCase .SCHEMA =>
66
- val schema = Dataset
67
- .ofRows(session, transformRelation(request.getSchema.getPlan.getRoot))
66
+ val rel = request.getSchema.getPlan.getRoot
67
+ val schema = sessionHolder
68
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
68
69
.schema
69
70
builder.setSchema(
70
71
proto.AnalyzePlanResponse .Schema
@@ -73,8 +74,9 @@ private[connect] class SparkConnectAnalyzeHandler(
73
74
.build())
74
75
75
76
case proto.AnalyzePlanRequest .AnalyzeCase .EXPLAIN =>
76
- val queryExecution = Dataset
77
- .ofRows(session, transformRelation(request.getExplain.getPlan.getRoot))
77
+ val rel = request.getExplain.getPlan.getRoot
78
+ val queryExecution = sessionHolder
79
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
78
80
.queryExecution
79
81
val explainString = request.getExplain.getExplainMode match {
80
82
case proto.AnalyzePlanRequest .Explain .ExplainMode .EXPLAIN_MODE_SIMPLE =>
@@ -96,8 +98,9 @@ private[connect] class SparkConnectAnalyzeHandler(
96
98
.build())
97
99
98
100
case proto.AnalyzePlanRequest .AnalyzeCase .TREE_STRING =>
99
- val schema = Dataset
100
- .ofRows(session, transformRelation(request.getTreeString.getPlan.getRoot))
101
+ val rel = request.getTreeString.getPlan.getRoot
102
+ val schema = sessionHolder
103
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
101
104
.schema
102
105
val treeString = if (request.getTreeString.hasLevel) {
103
106
schema.treeString(request.getTreeString.getLevel)
@@ -111,8 +114,9 @@ private[connect] class SparkConnectAnalyzeHandler(
111
114
.build())
112
115
113
116
case proto.AnalyzePlanRequest .AnalyzeCase .IS_LOCAL =>
114
- val isLocal = Dataset
115
- .ofRows(session, transformRelation(request.getIsLocal.getPlan.getRoot))
117
+ val rel = request.getIsLocal.getPlan.getRoot
118
+ val isLocal = sessionHolder
119
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
116
120
.isLocal
117
121
builder.setIsLocal(
118
122
proto.AnalyzePlanResponse .IsLocal
@@ -121,8 +125,9 @@ private[connect] class SparkConnectAnalyzeHandler(
121
125
.build())
122
126
123
127
case proto.AnalyzePlanRequest .AnalyzeCase .IS_STREAMING =>
124
- val isStreaming = Dataset
125
- .ofRows(session, transformRelation(request.getIsStreaming.getPlan.getRoot))
128
+ val rel = request.getIsStreaming.getPlan.getRoot
129
+ val isStreaming = sessionHolder
130
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
126
131
.isStreaming
127
132
builder.setIsStreaming(
128
133
proto.AnalyzePlanResponse .IsStreaming
@@ -131,8 +136,9 @@ private[connect] class SparkConnectAnalyzeHandler(
131
136
.build())
132
137
133
138
case proto.AnalyzePlanRequest .AnalyzeCase .INPUT_FILES =>
134
- val inputFiles = Dataset
135
- .ofRows(session, transformRelation(request.getInputFiles.getPlan.getRoot))
139
+ val rel = request.getInputFiles.getPlan.getRoot
140
+ val inputFiles = sessionHolder
141
+ .updatePlanCache(rel, Dataset .ofRows(session, transformRelation(rel)))
136
142
.inputFiles
137
143
builder.setInputFiles(
138
144
proto.AnalyzePlanResponse .InputFiles
@@ -156,29 +162,37 @@ private[connect] class SparkConnectAnalyzeHandler(
156
162
.build())
157
163
158
164
case proto.AnalyzePlanRequest .AnalyzeCase .SAME_SEMANTICS =>
159
- val target = Dataset .ofRows(
160
- session,
161
- transformRelation(request.getSameSemantics.getTargetPlan.getRoot ))
162
- val other = Dataset .ofRows(
163
- session,
164
- transformRelation(request.getSameSemantics.getOtherPlan.getRoot ))
165
+ val targetRel = request.getSameSemantics.getTargetPlan.getRoot
166
+ val target = sessionHolder
167
+ .updatePlanCache(targetRel, Dataset .ofRows(session, transformRelation(targetRel) ))
168
+ val otherRel = request.getSameSemantics.getOtherPlan.getRoot
169
+ val other = sessionHolder
170
+ .updatePlanCache(otherRel, Dataset .ofRows(session, transformRelation(otherRel) ))
165
171
builder.setSameSemantics(
166
172
proto.AnalyzePlanResponse .SameSemantics
167
173
.newBuilder()
168
174
.setResult(target.sameSemantics(other)))
169
175
170
176
case proto.AnalyzePlanRequest .AnalyzeCase .SEMANTIC_HASH =>
171
- val semanticHash = Dataset
172
- .ofRows(session, transformRelation(request.getSemanticHash.getPlan.getRoot))
177
+ val rel = request.getSemanticHash.getPlan.getRoot
178
+ val semanticHash = sessionHolder
179
+ .updatePlanCache(
180
+ rel,
181
+ Dataset
182
+ .ofRows(session, transformRelation(rel)))
173
183
.semanticHash()
174
184
builder.setSemanticHash(
175
185
proto.AnalyzePlanResponse .SemanticHash
176
186
.newBuilder()
177
187
.setResult(semanticHash))
178
188
179
189
case proto.AnalyzePlanRequest .AnalyzeCase .PERSIST =>
180
- val target = Dataset
181
- .ofRows(session, transformRelation(request.getPersist.getRelation))
190
+ val rel = request.getPersist.getRelation
191
+ val target = sessionHolder
192
+ .updatePlanCache(
193
+ rel,
194
+ Dataset
195
+ .ofRows(session, transformRelation(rel)))
182
196
if (request.getPersist.hasStorageLevel) {
183
197
target.persist(
184
198
StorageLevelProtoConverter .toStorageLevel(request.getPersist.getStorageLevel))
@@ -188,8 +202,12 @@ private[connect] class SparkConnectAnalyzeHandler(
188
202
builder.setPersist(proto.AnalyzePlanResponse .Persist .newBuilder().build())
189
203
190
204
case proto.AnalyzePlanRequest .AnalyzeCase .UNPERSIST =>
191
- val target = Dataset
192
- .ofRows(session, transformRelation(request.getUnpersist.getRelation))
205
+ val rel = request.getUnpersist.getRelation
206
+ val target = sessionHolder
207
+ .updatePlanCache(
208
+ rel,
209
+ Dataset
210
+ .ofRows(session, transformRelation(rel)))
193
211
if (request.getUnpersist.hasBlocking) {
194
212
target.unpersist(request.getUnpersist.getBlocking)
195
213
} else {
@@ -198,8 +216,12 @@ private[connect] class SparkConnectAnalyzeHandler(
198
216
builder.setUnpersist(proto.AnalyzePlanResponse .Unpersist .newBuilder().build())
199
217
200
218
case proto.AnalyzePlanRequest .AnalyzeCase .GET_STORAGE_LEVEL =>
201
- val target = Dataset
202
- .ofRows(session, transformRelation(request.getGetStorageLevel.getRelation))
219
+ val rel = request.getGetStorageLevel.getRelation
220
+ val target = sessionHolder
221
+ .updatePlanCache(
222
+ rel,
223
+ Dataset
224
+ .ofRows(session, transformRelation(rel)))
203
225
val storageLevel = target.storageLevel
204
226
builder.setGetStorageLevel(
205
227
proto.AnalyzePlanResponse .GetStorageLevel
0 commit comments