Skip to content

Commit 6cdbb08

Browse files
zhengruifengbeliefer
authored andcommitted
[SPARK-41005][COLLECT][FOLLOWUP] Remove JSON code path and use RDD.collect in Arrow code path
### What changes were proposed in this pull request? 1, Remove JSON code path; 2, use RDD.collect in Arrow code path, since existing tests were already broken in Arrow code path; 3, reenable `test_fill_na` ### Why are the changes needed? existing Arrow code path is still problematic and it fails and fallback to JSON code path, which change the output datatypes of `test_fill_na` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? reenabled test and added UT Closes apache#38706 from zhengruifeng/collect_disable_json. Authored-by: Ruifeng Zheng <ruifengz@apache.org> Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
1 parent f9a7d77 commit 6cdbb08

6 files changed

Lines changed: 82 additions & 240 deletions

File tree

connector/connect/src/main/protobuf/spark/connect/base.proto

Lines changed: 1 addition & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -139,11 +139,7 @@ message ExecutePlanRequest {
139139
message ExecutePlanResponse {
140140
string client_id = 1;
141141

142-
// Result type
143-
oneof result_type {
144-
ArrowBatch arrow_batch = 2;
145-
JSONBatch json_batch = 3;
146-
}
142+
ArrowBatch arrow_batch = 2;
147143

148144
// Metrics for the query execution. Typically, this field is only present in the last
149145
// batch of results and then represent the overall state of the query execution.
@@ -155,14 +151,6 @@ message ExecutePlanResponse {
155151
bytes data = 2;
156152
}
157153

158-
// Message type when the result is returned as JSON. This is essentially a bulk wrapper
159-
// for the JSON result of a Spark DataFrame. All rows are returned in the JSON record format
160-
// of `{col -> row}`.
161-
message JSONBatch {
162-
int64 row_count = 1;
163-
bytes data = 2;
164-
}
165-
166154
message Metrics {
167155

168156
repeated MetricObject metrics = 1;

connector/connect/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamHandler.scala

Lines changed: 11 additions & 145 deletions
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,10 @@
1818
package org.apache.spark.sql.connect.service
1919

2020
import scala.collection.JavaConverters._
21-
import scala.util.control.NonFatal
2221

2322
import com.google.protobuf.ByteString
2423
import io.grpc.stub.StreamObserver
2524

26-
import org.apache.spark.SparkException
2725
import org.apache.spark.connect.proto
2826
import org.apache.spark.connect.proto.{ExecutePlanRequest, ExecutePlanResponse}
2927
import org.apache.spark.internal.Logging
@@ -34,7 +32,6 @@ import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
3432
import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, AdaptiveSparkPlanHelper, QueryStageExec}
3533
import org.apache.spark.sql.execution.arrow.ArrowConverters
3634
import org.apache.spark.sql.types.StructType
37-
import org.apache.spark.util.ThreadUtils
3835

3936
class SparkConnectStreamHandler(responseObserver: StreamObserver[ExecutePlanResponse])
4037
extends Logging {
@@ -57,75 +54,7 @@ class SparkConnectStreamHandler(responseObserver: StreamObserver[ExecutePlanResp
5754
// Extract the plan from the request and convert it to a logical plan
5855
val planner = new SparkConnectPlanner(session)
5956
val dataframe = Dataset.ofRows(session, planner.transformRelation(request.getPlan.getRoot))
60-
try {
61-
processAsArrowBatches(request.getClientId, dataframe)
62-
} catch {
63-
case e: Exception =>
64-
logWarning(e.getMessage)
65-
processAsJsonBatches(request.getClientId, dataframe)
66-
}
67-
}
68-
69-
def processAsJsonBatches(clientId: String, dataframe: DataFrame): Unit = {
70-
// Only process up to 10MB of data.
71-
val sb = new StringBuilder
72-
var rowCount = 0
73-
dataframe.toJSON
74-
.collect()
75-
.foreach(row => {
76-
77-
// There are a few cases to cover here.
78-
// 1. The aggregated buffer size is larger than the MAX_BATCH_SIZE
79-
// -> send the current batch and reset.
80-
// 2. The aggregated buffer size is smaller than the MAX_BATCH_SIZE
81-
// -> append the row to the buffer.
82-
// 3. The row in question is larger than the MAX_BATCH_SIZE
83-
// -> fail the query.
84-
85-
// Case 3. - Fail
86-
if (row.size > MAX_BATCH_SIZE) {
87-
throw SparkException.internalError(
88-
s"Serialized row is larger than MAX_BATCH_SIZE: ${row.size} > ${MAX_BATCH_SIZE}")
89-
}
90-
91-
// Case 1 - FLush and send.
92-
if (sb.size + row.size > MAX_BATCH_SIZE) {
93-
val response = proto.ExecutePlanResponse.newBuilder().setClientId(clientId)
94-
val batch = proto.ExecutePlanResponse.JSONBatch
95-
.newBuilder()
96-
.setData(ByteString.copyFromUtf8(sb.toString()))
97-
.setRowCount(rowCount)
98-
.build()
99-
response.setJsonBatch(batch)
100-
responseObserver.onNext(response.build())
101-
sb.clear()
102-
sb.append(row)
103-
rowCount = 1
104-
} else {
105-
// Case 2 - Append.
106-
// Make sure to put the newline delimiters only between items and not at the end.
107-
if (rowCount > 0) {
108-
sb.append("\n")
109-
}
110-
sb.append(row)
111-
rowCount += 1
112-
}
113-
})
114-
115-
// If the last batch is not empty, send out the data to the client.
116-
if (sb.size > 0) {
117-
val response = proto.ExecutePlanResponse.newBuilder().setClientId(clientId)
118-
val batch = proto.ExecutePlanResponse.JSONBatch
119-
.newBuilder()
120-
.setData(ByteString.copyFromUtf8(sb.toString()))
121-
.setRowCount(rowCount)
122-
.build()
123-
response.setJsonBatch(batch)
124-
responseObserver.onNext(response.build())
125-
}
126-
127-
responseObserver.onNext(sendMetricsToResponse(clientId, dataframe))
128-
responseObserver.onCompleted()
57+
processAsArrowBatches(request.getClientId, dataframe)
12958
}
13059

13160
def processAsArrowBatches(clientId: String, dataframe: DataFrame): Unit = {
@@ -142,83 +71,20 @@ class SparkConnectStreamHandler(responseObserver: StreamObserver[ExecutePlanResp
14271
var numSent = 0
14372

14473
if (numPartitions > 0) {
145-
type Batch = (Array[Byte], Long)
146-
14774
val batches = rows.mapPartitionsInternal(
14875
SparkConnectStreamHandler
14976
.rowToArrowConverter(schema, maxRecordsPerBatch, maxBatchSize, timeZoneId))
15077

151-
val signal = new Object
152-
val partitions = collection.mutable.Map.empty[Int, Array[Batch]]
153-
var error: Throwable = null
154-
155-
val processPartition = (iter: Iterator[Batch]) => iter.toArray
156-
157-
// This callback is executed by the DAGScheduler thread.
158-
// After fetching a partition, it inserts the partition into the Map, and then
159-
// wakes up the main thread.
160-
val resultHandler = (partitionId: Int, partition: Array[Batch]) => {
161-
signal.synchronized {
162-
partitions(partitionId) = partition
163-
signal.notify()
164-
}
165-
()
166-
}
167-
168-
val future = spark.sparkContext.submitJob(
169-
rdd = batches,
170-
processPartition = processPartition,
171-
partitions = Seq.range(0, numPartitions),
172-
resultHandler = resultHandler,
173-
resultFunc = () => ())
174-
175-
// Collect errors and propagate them to the main thread.
176-
future.onComplete { result =>
177-
result.failed.foreach { throwable =>
178-
signal.synchronized {
179-
error = throwable
180-
signal.notify()
181-
}
182-
}
183-
}(ThreadUtils.sameThread)
184-
185-
// The main thread will wait until 0-th partition is available,
186-
// then send it to client and wait for the next partition.
187-
// Different from the implementation of [[Dataset#collectAsArrowToPython]], it sends
188-
// the arrow batches in main thread to avoid DAGScheduler thread been blocked for
189-
// tasks not related to scheduling. This is particularly important if there are
190-
// multiple users or clients running code at the same time.
191-
var currentPartitionId = 0
192-
while (currentPartitionId < numPartitions) {
193-
val partition = signal.synchronized {
194-
var result = partitions.remove(currentPartitionId)
195-
while (result.isEmpty && error == null) {
196-
signal.wait()
197-
result = partitions.remove(currentPartitionId)
198-
}
199-
error match {
200-
case NonFatal(e) =>
201-
responseObserver.onError(error)
202-
logError("Error while processing query.", e)
203-
return
204-
case fatal: Throwable => throw fatal
205-
case null => result.get
206-
}
207-
}
208-
209-
partition.foreach { case (bytes, count) =>
210-
val response = proto.ExecutePlanResponse.newBuilder().setClientId(clientId)
211-
val batch = proto.ExecutePlanResponse.ArrowBatch
212-
.newBuilder()
213-
.setRowCount(count)
214-
.setData(ByteString.copyFrom(bytes))
215-
.build()
216-
response.setArrowBatch(batch)
217-
responseObserver.onNext(response.build())
218-
numSent += 1
219-
}
220-
221-
currentPartitionId += 1
78+
batches.collect().foreach { case (bytes, count) =>
79+
val response = proto.ExecutePlanResponse.newBuilder().setClientId(clientId)
80+
val batch = proto.ExecutePlanResponse.ArrowBatch
81+
.newBuilder()
82+
.setRowCount(count)
83+
.setData(ByteString.copyFrom(bytes))
84+
.build()
85+
response.setArrowBatch(batch)
86+
responseObserver.onNext(response.build())
87+
numSent += 1
22288
}
22389
}
22490

python/pyspark/sql/connect/client.py

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,6 @@
1616
#
1717

1818

19-
import io
2019
import logging
2120
import os
2221
import typing
@@ -446,13 +445,9 @@ def _analyze(self, plan: pb2.Plan, explain_mode: str = "extended") -> AnalyzeRes
446445
return AnalyzeResult.fromProto(resp)
447446

448447
def _process_batch(self, b: pb2.ExecutePlanResponse) -> Optional[pandas.DataFrame]:
449-
import pandas as pd
450-
451448
if b.arrow_batch is not None and len(b.arrow_batch.data) > 0:
452449
with pa.ipc.open_stream(b.arrow_batch.data) as rd:
453450
return rd.read_pandas()
454-
elif b.json_batch is not None and len(b.json_batch.data) > 0:
455-
return pd.read_json(io.BytesIO(b.json_batch.data), lines=True)
456451
return None
457452

458453
def _execute_and_fetch(self, req: pb2.ExecutePlanRequest) -> typing.Optional[pandas.DataFrame]:

python/pyspark/sql/connect/proto/base_pb2.py

Lines changed: 14 additions & 27 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@
3636

3737

3838
DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
39-
b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x01\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\xad\x07\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12M\n\njson_batch\x18\x03 \x01(\x0b\x32,.spark.connect.ExecutePlanResponse.JSONBatchH\x00R\tjsonBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a<\n\tJSONBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricTypeB\r\n\x0bresult_type2\xc7\x01\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3'
39+
b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"\xb5\x01\n\x07\x45xplain\x12\x45\n\x0c\x65xplain_mode\x18\x01 \x01(\x0e\x32".spark.connect.Explain.ExplainModeR\x0b\x65xplainMode"c\n\x0b\x45xplainMode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\n\n\x06SIMPLE\x10\x01\x12\x0c\n\x08\x45XTENDED\x10\x02\x12\x0b\n\x07\x43ODEGEN\x10\x03\x12\x08\n\x04\x43OST\x10\x04\x12\r\n\tFORMATTED\x10\x05"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\x81\x02\n\x12\x41nalyzePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\x30\n\x07\x65xplain\x18\x05 \x01(\x0b\x32\x16.spark.connect.ExplainR\x07\x65xplainB\x0e\n\x0c_client_type"\x8a\x01\n\x13\x41nalyzePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x12%\n\x0e\x65xplain_string\x18\x03 \x01(\tR\rexplainString"\xcf\x01\n\x12\x45xecutePlanRequest\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"\x8f\x06\n\x13\x45xecutePlanResponse\x12\x1b\n\tclient_id\x18\x01 \x01(\tR\x08\x63lientId\x12N\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchR\narrowBatch\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x1a=\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType2\xc7\x01\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x42"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3'
4040
)
4141

4242

@@ -48,7 +48,6 @@
4848
_EXECUTEPLANREQUEST = DESCRIPTOR.message_types_by_name["ExecutePlanRequest"]
4949
_EXECUTEPLANRESPONSE = DESCRIPTOR.message_types_by_name["ExecutePlanResponse"]
5050
_EXECUTEPLANRESPONSE_ARROWBATCH = _EXECUTEPLANRESPONSE.nested_types_by_name["ArrowBatch"]
51-
_EXECUTEPLANRESPONSE_JSONBATCH = _EXECUTEPLANRESPONSE.nested_types_by_name["JSONBatch"]
5251
_EXECUTEPLANRESPONSE_METRICS = _EXECUTEPLANRESPONSE.nested_types_by_name["Metrics"]
5352
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT = _EXECUTEPLANRESPONSE_METRICS.nested_types_by_name[
5453
"MetricObject"
@@ -139,15 +138,6 @@
139138
# @@protoc_insertion_point(class_scope:spark.connect.ExecutePlanResponse.ArrowBatch)
140139
},
141140
),
142-
"JSONBatch": _reflection.GeneratedProtocolMessageType(
143-
"JSONBatch",
144-
(_message.Message,),
145-
{
146-
"DESCRIPTOR": _EXECUTEPLANRESPONSE_JSONBATCH,
147-
"__module__": "spark.connect.base_pb2"
148-
# @@protoc_insertion_point(class_scope:spark.connect.ExecutePlanResponse.JSONBatch)
149-
},
150-
),
151141
"Metrics": _reflection.GeneratedProtocolMessageType(
152142
"Metrics",
153143
(_message.Message,),
@@ -191,7 +181,6 @@
191181
)
192182
_sym_db.RegisterMessage(ExecutePlanResponse)
193183
_sym_db.RegisterMessage(ExecutePlanResponse.ArrowBatch)
194-
_sym_db.RegisterMessage(ExecutePlanResponse.JSONBatch)
195184
_sym_db.RegisterMessage(ExecutePlanResponse.Metrics)
196185
_sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricObject)
197186
_sym_db.RegisterMessage(ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntry)
@@ -219,19 +208,17 @@
219208
_EXECUTEPLANREQUEST._serialized_start = 986
220209
_EXECUTEPLANREQUEST._serialized_end = 1193
221210
_EXECUTEPLANRESPONSE._serialized_start = 1196
222-
_EXECUTEPLANRESPONSE._serialized_end = 2137
223-
_EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 1479
224-
_EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 1540
225-
_EXECUTEPLANRESPONSE_JSONBATCH._serialized_start = 1542
226-
_EXECUTEPLANRESPONSE_JSONBATCH._serialized_end = 1602
227-
_EXECUTEPLANRESPONSE_METRICS._serialized_start = 1605
228-
_EXECUTEPLANRESPONSE_METRICS._serialized_end = 2122
229-
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 1700
230-
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 2032
231-
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 1909
232-
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 2032
233-
_EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 2034
234-
_EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 2122
235-
_SPARKCONNECTSERVICE._serialized_start = 2140
236-
_SPARKCONNECTSERVICE._serialized_end = 2339
211+
_EXECUTEPLANRESPONSE._serialized_end = 1979
212+
_EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 1398
213+
_EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 1459
214+
_EXECUTEPLANRESPONSE_METRICS._serialized_start = 1462
215+
_EXECUTEPLANRESPONSE_METRICS._serialized_end = 1979
216+
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 1557
217+
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 1889
218+
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 1766
219+
_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 1889
220+
_EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 1891
221+
_EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 1979
222+
_SPARKCONNECTSERVICE._serialized_start = 1982
223+
_SPARKCONNECTSERVICE._serialized_end = 2181
237224
# @@protoc_insertion_point(module_scope)

0 commit comments

Comments
 (0)