From 24f90b29bb816896d954e8404987584af4b4b1f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 12:00:33 -0300 Subject: [PATCH 01/12] proto: add GetNumPartitions to AnalyzePlanRequest/Response --- .../src/main/protobuf/spark/connect/base.proto | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index c7247129f1907..a7ed9b26899b3 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -114,6 +114,7 @@ message AnalyzePlanRequest { Persist persist = 14; Unpersist unpersist = 15; GetStorageLevel get_storage_level = 16; + GetNumPartitions get_num_partitions = 17; JsonToDDL json_to_ddl = 18; } @@ -221,6 +222,11 @@ message AnalyzePlanRequest { Relation relation = 1; } + message GetNumPartitions { + // (Required) The logical plan to be analyzed. + Plan plan = 1; + } + message JsonToDDL { // (Required) The JSON formatted string to be converted to DDL. string json_string = 1; @@ -229,7 +235,7 @@ message AnalyzePlanRequest { // Response to performing analysis of the query. Contains relevant metadata to be able to // reason about the performance. -// Next ID: 16 +// Next ID: 17 message AnalyzePlanResponse { string session_id = 1; // Server-side generated idempotency key that the client can use to assert that the server side @@ -250,6 +256,7 @@ message AnalyzePlanResponse { Persist persist = 12; Unpersist unpersist = 13; GetStorageLevel get_storage_level = 14; + GetNumPartitions get_num_partitions = 15; JsonToDDL json_to_ddl = 16; } @@ -303,6 +310,11 @@ message AnalyzePlanResponse { StorageLevel storage_level = 1; } + message GetNumPartitions { + // The number of partitions in the physical execution plan. + int32 num_partitions = 1; + } + message JsonToDDL { string ddl_string = 1; } From 5bdb1c7df03ab5035e7f25e3941501096ecbf1c3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 12:02:43 -0300 Subject: [PATCH 02/12] test: add getNumPartitions assertions to Dataset inspection E2E test --- .../scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala index 0e8cb9348c7f8..a238bde06b879 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/ClientE2ETestSuite.scala @@ -629,6 +629,8 @@ class ClientE2ETestSuite assert(!df.isStreaming) assert(df.toString.contains("[id: bigint]")) assert(df.inputFiles.isEmpty) + assert(df.repartition(4).getNumPartitions === 4) + assert(df.coalesce(1).getNumPartitions === 1) } test("Dataset schema") { From 18b05f239f26ac720a70d44115078b7445686de1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 12:05:13 -0300 Subject: [PATCH 03/12] feat: add Dataset.getNumPartitions client-side implementation --- .../scala/org/apache/spark/sql/connect/Dataset.scala | 11 +++++++++++ .../spark/sql/connect/client/SparkConnectClient.scala | 7 +++++++ 2 files changed, 18 insertions(+) diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Dataset.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Dataset.scala index b57ea66bb1f7d..fee4b9fcc9861 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Dataset.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/Dataset.scala @@ -257,6 +257,17 @@ class Dataset[T] private[sql] ( .getIsStreaming .getIsStreaming + /** + * Returns the number of partitions of this Dataset. + * + * @group basic + * @since 4.2.0 + */ + def getNumPartitions: Int = sparkSession + .analyze(plan, proto.AnalyzePlanRequest.AnalyzeCase.GET_NUM_PARTITIONS) + .getGetNumPartitions + .getNumPartitions + /** @inheritdoc */ // scalastyle:off println def show(numRows: Int, truncate: Boolean): Unit = { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index d9b9ba35b5e6c..7bedaef95673e 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -408,6 +408,13 @@ private[sql] class SparkConnectClient( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.SPARK_VERSION => builder.setSparkVersion(proto.AnalyzePlanRequest.SparkVersion.newBuilder().build()) + case proto.AnalyzePlanRequest.AnalyzeCase.GET_NUM_PARTITIONS => + assert(maybeCompressedPlan.isDefined) + builder.setGetNumPartitions( + proto.AnalyzePlanRequest.GetNumPartitions + .newBuilder() + .setPlan(maybeCompressedPlan.get) + .build()) case other => throw new IllegalArgumentException(s"Unknown Analyze request $other") } analyze(builder) From e0d787babe3c41cbd2fd214ff9ba05c5abef3513 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 12:07:40 -0300 Subject: [PATCH 04/12] feat: add server-side handler for Dataset.getNumPartitions --- .../service/RequestDecompressionInterceptor.scala | 10 ++++++++++ .../connect/service/SparkConnectAnalyzeHandler.scala | 10 ++++++++++ 2 files changed, 20 insertions(+) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/RequestDecompressionInterceptor.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/RequestDecompressionInterceptor.scala index d93dc2069cf6f..edae9b0b663ef 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/RequestDecompressionInterceptor.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/RequestDecompressionInterceptor.scala @@ -193,6 +193,16 @@ class RequestDecompressionInterceptor extends ServerInterceptor with Logging { .build()) (req, Seq(size)) + case proto.AnalyzePlanRequest.AnalyzeCase.GET_NUM_PARTITIONS => + val (req, size) = decompress( + request, + request.getGetNumPartitions.getPlan, + p => + request.toBuilder + .setGetNumPartitions(request.getGetNumPartitions.toBuilder.setPlan(p)) + .build()) + (req, Seq(size)) + case proto.AnalyzePlanRequest.AnalyzeCase.SEMANTIC_HASH => val (req, size) = decompress( request, diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index 49450f9387198..faceba06a8428 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -240,6 +240,16 @@ private[connect] class SparkConnectAnalyzeHandler( .setDdlString(ddl) .build()) + case proto.AnalyzePlanRequest.AnalyzeCase.GET_NUM_PARTITIONS => + val rel = transformRelationPlan(request.getGetNumPartitions.getPlan) + val numPartitions = + getDataFrameWithoutExecuting(rel).queryExecution.executedPlan.execute().getNumPartitions + builder.setGetNumPartitions( + proto.AnalyzePlanResponse.GetNumPartitions + .newBuilder() + .setNumPartitions(numPartitions) + .build()) + // NOTE: When adding a new AnalyzePlanRequest case here, also update // RequestDecompressionInterceptor.decompressAnalyzePlanRequest() to handle // this case. The interceptor has a default case that throws UnsupportedOperationException From e4db9c91d8f5a1377835cea59a5a2651ee1263b9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 12:23:54 -0300 Subject: [PATCH 05/12] feat: add Python client getNumPartitions for Spark Connect DataFrame --- python/pyspark/sql/connect/client/core.py | 8 ++++++++ python/pyspark/sql/connect/dataframe.py | 6 ++++++ python/pyspark/sql/tests/connect/test_connect_basic.py | 4 ++++ 3 files changed, 18 insertions(+) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 925b114d070dd..d9ff69cb1c8ec 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -539,6 +539,7 @@ def __init__( semantic_hash: Optional[int], storage_level: Optional[StorageLevel], ddl_string: Optional[str], + num_partitions: Optional[int], ): self.schema = schema self.explain_string = explain_string @@ -552,6 +553,7 @@ def __init__( self.semantic_hash = semantic_hash self.storage_level = storage_level self.ddl_string = ddl_string + self.num_partitions = num_partitions @classmethod def fromProto(cls, pb: Any) -> "AnalyzeResult": @@ -567,6 +569,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": semantic_hash: Optional[int] = None storage_level: Optional[StorageLevel] = None ddl_string: Optional[str] = None + num_partitions: Optional[int] = None if pb.HasField("schema"): schema = types.proto_schema_to_pyspark_data_type(pb.schema.schema) @@ -596,6 +599,8 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": storage_level = proto_to_storage_level(pb.get_storage_level.storage_level) elif pb.HasField("json_to_ddl"): ddl_string = pb.json_to_ddl.ddl_string + elif pb.HasField("get_num_partitions"): + num_partitions = pb.get_num_partitions.num_partitions else: raise SparkConnectException("No analyze result found!") @@ -612,6 +617,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": semantic_hash, storage_level, ddl_string, + num_partitions, ) @@ -1440,6 +1446,8 @@ def _analyze(self, method: str, **kwargs: Any) -> AnalyzeResult: req.get_storage_level.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) elif method == "json_to_ddl": req.json_to_ddl.json_string = cast(str, kwargs.get("json_string")) + elif method == "get_num_partitions": + req.get_num_partitions.plan.CopyFrom(cast(pb2.Plan, kwargs.get("plan"))) else: raise PySparkValueError( errorClass="UNSUPPORTED_OPERATION", diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index c6602e08fac4c..e5226638086ef 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -2044,6 +2044,12 @@ def inputFiles(self) -> List[str]: assert result is not None return result + def getNumPartitions(self) -> int: + query = self._plan.to_proto(self._session.client) + result = self._session.client._analyze(method="get_num_partitions", plan=query).num_partitions + assert result is not None + return result + def to(self, schema: StructType) -> ParentDataFrame: assert schema is not None res = DataFrame( diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 516c3ad375177..dbfcc23368583 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -613,6 +613,10 @@ def test_is_streaming(self): self.assertFalse(self.connect.read.table(self.tbl_name).isStreaming) self.assertFalse(self.connect.sql("SELECT 1 AS X LIMIT 0").isStreaming) + def test_get_num_partitions(self): + self.assertEqual(self.connect.range(10).repartition(4).getNumPartitions(), 4) + self.assertEqual(self.connect.range(10).coalesce(1).getNumPartitions(), 1) + def test_input_files(self): # SPARK-41216: Test input files tmpPath = tempfile.mkdtemp() From a1097f63a9f8809c2424bab7cbe2a46d5f9811b1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 13:22:38 -0300 Subject: [PATCH 06/12] fix: correct proto field numbers (19 for request, 17 for response) --- .../common/src/main/protobuf/spark/connect/base.proto | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index a7ed9b26899b3..cf6e26bd981e0 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -114,8 +114,8 @@ message AnalyzePlanRequest { Persist persist = 14; Unpersist unpersist = 15; GetStorageLevel get_storage_level = 16; - GetNumPartitions get_num_partitions = 17; JsonToDDL json_to_ddl = 18; + GetNumPartitions get_num_partitions = 19; } message Schema { @@ -235,7 +235,7 @@ message AnalyzePlanRequest { // Response to performing analysis of the query. Contains relevant metadata to be able to // reason about the performance. -// Next ID: 17 +// Next ID: 18 message AnalyzePlanResponse { string session_id = 1; // Server-side generated idempotency key that the client can use to assert that the server side @@ -256,8 +256,8 @@ message AnalyzePlanResponse { Persist persist = 12; Unpersist unpersist = 13; GetStorageLevel get_storage_level = 14; - GetNumPartitions get_num_partitions = 15; JsonToDDL json_to_ddl = 16; + GetNumPartitions get_num_partitions = 17; } message Schema { From 4b8f9873c564912a0c525ab52ff04a8ad5cb0c7f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 14:11:45 -0300 Subject: [PATCH 07/12] test: add server unit test for getNumPartitions in SparkConnectServiceSuite --- .../planner/SparkConnectServiceSuite.scala | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala index 8ac46b25a5f3e..9a4e1e9c5f790 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala @@ -161,6 +161,29 @@ class SparkConnectServiceSuite val response6 = handler.process(request6, sparkSessionHolder) assert(response6.hasInputFiles) assert(response6.getInputFiles.getFilesCount === 0) + + val repartitionPlan = proto.Plan + .newBuilder() + .setRoot( + proto.Relation + .newBuilder() + .setRepartition( + proto.Repartition + .newBuilder() + .setInput(plan.getRoot) + .setNumPartitions(4) + .setShuffle(true) + .build()) + .build()) + .build() + val request7 = proto.AnalyzePlanRequest + .newBuilder() + .setGetNumPartitions( + proto.AnalyzePlanRequest.GetNumPartitions.newBuilder().setPlan(repartitionPlan).build()) + .build() + val response7 = handler.process(request7, sparkSessionHolder) + assert(response7.hasGetNumPartitions) + assert(response7.getGetNumPartitions.getNumPartitions === 4) } } From be7029fc222947151e7ecb39eeefe108fc17926a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 16:31:53 -0300 Subject: [PATCH 08/12] Trigger Build From b357ef6dff08f7e3ecf017f23c95c9149761a87a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 17:41:52 -0300 Subject: [PATCH 09/12] Formatting + generating new protos --- python/pyspark/sql/connect/dataframe.py | 4 +- python/pyspark/sql/connect/proto/base_pb2.py | 396 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 50 +++ 3 files changed, 253 insertions(+), 197 deletions(-) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index e5226638086ef..2fa39bd916f42 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -2046,7 +2046,9 @@ def inputFiles(self) -> List[str]: def getNumPartitions(self) -> int: query = self._plan.to_proto(self._session.client) - result = self._session.client._analyze(method="get_num_partitions", plan=query).num_partitions + result = self._session.client._analyze( + method="get_num_partitions", plan=query + ).num_partitions assert result is not None return result diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index a77c61ca6d2b4..19fcee39ebb9b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -46,7 +46,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto\x1a\x1dspark/connect/pipelines.proto"\xe3\x03\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\x63ommand\x12\\\n\x14\x63ompressed_operation\x18\x03 \x01(\x0b\x32\'.spark.connect.Plan.CompressedOperationH\x00R\x13\x63ompressedOperation\x1a\x8e\x02\n\x13\x43ompressedOperation\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12G\n\x07op_type\x18\x02 \x01(\x0e\x32..spark.connect.Plan.CompressedOperation.OpTypeR\x06opType\x12L\n\x11\x63ompression_codec\x18\x03 \x01(\x0e\x32\x1f.spark.connect.CompressionCodecR\x10\x63ompressionCodec"L\n\x06OpType\x12\x17\n\x13OP_TYPE_UNSPECIFIED\x10\x00\x12\x14\n\x10OP_TYPE_RELATION\x10\x01\x12\x13\n\x0fOP_TYPE_COMMAND\x10\x02\x42\t\n\x07op_type"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"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\x83\x06\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\x85\x02\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12^\n\x17result_chunking_options\x18\x02 \x01(\x0b\x32$.spark.connect.ResultChunkingOptionsH\x00R\x15resultChunkingOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x87\x1c\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12X\n\x15pipeline_event_result\x18\x15 \x01(\x0b\x32".spark.connect.PipelineEventResultH\x00R\x13pipelineEventResult\x12^\n\x17pipeline_command_result\x18\x16 \x01(\x0b\x32$.spark.connect.PipelineCommandResultH\x00R\x15pipelineCommandResult\x12\x8d\x01\n(pipeline_query_function_execution_signal\x18\x17 \x01(\x0b\x32\x33.spark.connect.PipelineQueryFunctionExecutionSignalH\x00R$pipelineQueryFunctionExecutionSignal\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a\xf8\x01\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\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x12$\n\x0b\x63hunk_index\x18\x04 \x01(\x03H\x01R\nchunkIndex\x88\x01\x01\x12\x32\n\x13num_chunks_in_batch\x18\x05 \x01(\x03H\x02R\x10numChunksInBatch\x88\x01\x01\x42\x0f\n\r_start_offsetB\x0e\n\x0c_chunk_indexB\x16\n\x14_num_chunks_in_batch\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\nmetricType\x1a\x93\x02\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x12)\n\x0eroot_error_idx\x18\x05 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x06 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrorsB\x11\n\x0f_root_error_idx\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\xb5\x01\n\x15ResultChunkingOptions\x12;\n\x1a\x61llow_arrow_batch_chunking\x18\x01 \x01(\x08R\x17\x61llowArrowBatchChunking\x12@\n\x1apreferred_arrow_chunk_size\x18\x02 \x01(\x03H\x00R\x17preferredArrowChunkSize\x88\x01\x01\x42\x1d\n\x1b_preferred_arrow_chunk_size"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xd9\x0f\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\xa6\x04\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x12r\n\x14\x62reaking_change_info\x18\x05 \x01(\x0b\x32;.spark.connect.FetchErrorDetailsResponse.BreakingChangeInfoH\x02R\x12\x62reakingChangeInfo\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_stateB\x17\n\x15_breaking_change_info\x1a\xfa\x01\n\x12\x42reakingChangeInfo\x12+\n\x11migration_message\x18\x01 \x03(\tR\x10migrationMessage\x12k\n\x11mitigation_config\x18\x02 \x01(\x0b\x32\x39.spark.connect.FetchErrorDetailsResponse.MitigationConfigH\x00R\x10mitigationConfig\x88\x01\x01\x12$\n\x0bneeds_audit\x18\x03 \x01(\x08H\x01R\nneedsAudit\x88\x01\x01\x42\x14\n\x12_mitigation_configB\x0e\n\x0c_needs_audit\x1a:\n\x10MitigationConfig\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xea\x02\n\x13\x43loneSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12)\n\x0enew_session_id\x18\x04 \x01(\tH\x02R\x0cnewSessionId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x11\n\x0f_new_session_id"\xcc\x01\n\x14\x43loneSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12$\n\x0enew_session_id\x18\x03 \x01(\tR\x0cnewSessionId\x12:\n\x1anew_server_side_session_id\x18\x04 \x01(\tR\x16newServerSideSessionId"\xd3\x04\n\x10GetStatusRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12V\n&client_observed_server_side_session_id\x18\x04 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12\x66\n\x10operation_status\x18\x05 \x01(\x0b\x32\x36.spark.connect.GetStatusRequest.OperationStatusRequestH\x02R\x0foperationStatus\x88\x01\x01\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions\x1at\n\x16OperationStatusRequest\x12#\n\roperation_ids\x18\x01 \x03(\tR\x0coperationIds\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensionsB\x0e\n\x0c_client_typeB)\n\'_client_observed_server_side_session_idB\x13\n\x11_operation_status"\xad\x05\n\x11GetStatusResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12_\n\x12operation_statuses\x18\x03 \x03(\x0b\x32\x30.spark.connect.GetStatusResponse.OperationStatusR\x11operationStatuses\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions\x1a\xab\x03\n\x0fOperationStatus\x12!\n\x0coperation_id\x18\x01 \x01(\tR\x0boperationId\x12U\n\x05state\x18\x02 \x01(\x0e\x32?.spark.connect.GetStatusResponse.OperationStatus.OperationStateR\x05state\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xe6\x01\n\x0eOperationState\x12\x1f\n\x1bOPERATION_STATE_UNSPECIFIED\x10\x00\x12\x1b\n\x17OPERATION_STATE_UNKNOWN\x10\x01\x12\x1b\n\x17OPERATION_STATE_RUNNING\x10\x02\x12\x1f\n\x1bOPERATION_STATE_TERMINATING\x10\x03\x12\x1d\n\x19OPERATION_STATE_SUCCEEDED\x10\x04\x12\x1a\n\x16OPERATION_STATE_FAILED\x10\x05\x12\x1d\n\x19OPERATION_STATE_CANCELLED\x10\x06*Q\n\x10\x43ompressionCodec\x12!\n\x1d\x43OMPRESSION_CODEC_UNSPECIFIED\x10\x00\x12\x1a\n\x16\x43OMPRESSION_CODEC_ZSTD\x10\x01\x32\xdf\x08\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\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x12Y\n\x0c\x43loneSession\x12".spark.connect.CloneSessionRequest\x1a#.spark.connect.CloneSessionResponse"\x00\x12P\n\tGetStatus\x12\x1f.spark.connect.GetStatusRequest\x1a .spark.connect.GetStatusResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto\x1a\x16spark/connect/ml.proto\x1a\x1dspark/connect/pipelines.proto"\xe3\x03\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\x63ommand\x12\\\n\x14\x63ompressed_operation\x18\x03 \x01(\x0b\x32\'.spark.connect.Plan.CompressedOperationH\x00R\x13\x63ompressedOperation\x1a\x8e\x02\n\x13\x43ompressedOperation\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12G\n\x07op_type\x18\x02 \x01(\x0e\x32..spark.connect.Plan.CompressedOperation.OpTypeR\x06opType\x12L\n\x11\x63ompression_codec\x18\x03 \x01(\x0e\x32\x1f.spark.connect.CompressionCodecR\x10\x63ompressionCodec"L\n\x06OpType\x12\x17\n\x13OP_TYPE_UNSPECIFIED\x10\x00\x12\x14\n\x10OP_TYPE_RELATION\x10\x01\x12\x13\n\x0fOP_TYPE_COMMAND\x10\x02\x42\t\n\x07op_type"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"\x96\x16\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x12\x62\n\x12get_num_partitions\x18\x13 \x01(\x0b\x32\x32.spark.connect.AnalyzePlanRequest.GetNumPartitionsH\x00R\x10getNumPartitions\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a;\n\x10GetNumPartitions\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xea\x0f\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x12\x63\n\x12get_num_partitions\x18\x11 \x01(\x0b\x32\x33.spark.connect.AnalyzePlanResponse.GetNumPartitionsH\x00R\x10getNumPartitions\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a\x39\n\x10GetNumPartitions\x12%\n\x0enum_partitions\x18\x01 \x01(\x05R\rnumPartitions\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\x83\x06\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\x85\x02\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12^\n\x17result_chunking_options\x18\x02 \x01(\x0b\x32$.spark.connect.ResultChunkingOptionsH\x00R\x15resultChunkingOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\x87\x1c\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12L\n\x11ml_command_result\x18\x14 \x01(\x0b\x32\x1e.spark.connect.MlCommandResultH\x00R\x0fmlCommandResult\x12X\n\x15pipeline_event_result\x18\x15 \x01(\x0b\x32".spark.connect.PipelineEventResultH\x00R\x13pipelineEventResult\x12^\n\x17pipeline_command_result\x18\x16 \x01(\x0b\x32$.spark.connect.PipelineCommandResultH\x00R\x15pipelineCommandResult\x12\x8d\x01\n(pipeline_query_function_execution_signal\x18\x17 \x01(\x0b\x32\x33.spark.connect.PipelineQueryFunctionExecutionSignalH\x00R$pipelineQueryFunctionExecutionSignal\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a\xf8\x01\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\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x12$\n\x0b\x63hunk_index\x18\x04 \x01(\x03H\x01R\nchunkIndex\x88\x01\x01\x12\x32\n\x13num_chunks_in_batch\x18\x05 \x01(\x03H\x02R\x10numChunksInBatch\x88\x01\x01\x42\x0f\n\r_start_offsetB\x0e\n\x0c_chunk_indexB\x16\n\x14_num_chunks_in_batch\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\nmetricType\x1a\x93\x02\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x12)\n\x0eroot_error_idx\x18\x05 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x06 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrorsB\x11\n\x0f_root_error_idx\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\xb5\x01\n\x15ResultChunkingOptions\x12;\n\x1a\x61llow_arrow_batch_chunking\x18\x01 \x01(\x08R\x17\x61llowArrowBatchChunking\x12@\n\x1apreferred_arrow_chunk_size\x18\x02 \x01(\x03H\x00R\x17preferredArrowChunkSize\x88\x01\x01\x42\x1d\n\x1b_preferred_arrow_chunk_size"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xd9\x0f\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\xa6\x04\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x12r\n\x14\x62reaking_change_info\x18\x05 \x01(\x0b\x32;.spark.connect.FetchErrorDetailsResponse.BreakingChangeInfoH\x02R\x12\x62reakingChangeInfo\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_stateB\x17\n\x15_breaking_change_info\x1a\xfa\x01\n\x12\x42reakingChangeInfo\x12+\n\x11migration_message\x18\x01 \x03(\tR\x10migrationMessage\x12k\n\x11mitigation_config\x18\x02 \x01(\x0b\x32\x39.spark.connect.FetchErrorDetailsResponse.MitigationConfigH\x00R\x10mitigationConfig\x88\x01\x01\x12$\n\x0bneeds_audit\x18\x03 \x01(\x08H\x01R\nneedsAudit\x88\x01\x01\x42\x14\n\x12_mitigation_configB\x0e\n\x0c_needs_audit\x1a:\n\x10MitigationConfig\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xea\x02\n\x13\x43loneSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12)\n\x0enew_session_id\x18\x04 \x01(\tH\x02R\x0cnewSessionId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x11\n\x0f_new_session_id"\xcc\x01\n\x14\x43loneSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12$\n\x0enew_session_id\x18\x03 \x01(\tR\x0cnewSessionId\x12:\n\x1anew_server_side_session_id\x18\x04 \x01(\tR\x16newServerSideSessionId"\xd3\x04\n\x10GetStatusRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12V\n&client_observed_server_side_session_id\x18\x04 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12\x66\n\x10operation_status\x18\x05 \x01(\x0b\x32\x36.spark.connect.GetStatusRequest.OperationStatusRequestH\x02R\x0foperationStatus\x88\x01\x01\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions\x1at\n\x16OperationStatusRequest\x12#\n\roperation_ids\x18\x01 \x03(\tR\x0coperationIds\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensionsB\x0e\n\x0c_client_typeB)\n\'_client_observed_server_side_session_idB\x13\n\x11_operation_status"\xad\x05\n\x11GetStatusResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12_\n\x12operation_statuses\x18\x03 \x03(\x0b\x32\x30.spark.connect.GetStatusResponse.OperationStatusR\x11operationStatuses\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions\x1a\xab\x03\n\x0fOperationStatus\x12!\n\x0coperation_id\x18\x01 \x01(\tR\x0boperationId\x12U\n\x05state\x18\x02 \x01(\x0e\x32?.spark.connect.GetStatusResponse.OperationStatus.OperationStateR\x05state\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xe6\x01\n\x0eOperationState\x12\x1f\n\x1bOPERATION_STATE_UNSPECIFIED\x10\x00\x12\x1b\n\x17OPERATION_STATE_UNKNOWN\x10\x01\x12\x1b\n\x17OPERATION_STATE_RUNNING\x10\x02\x12\x1f\n\x1bOPERATION_STATE_TERMINATING\x10\x03\x12\x1d\n\x19OPERATION_STATE_SUCCEEDED\x10\x04\x12\x1a\n\x16OPERATION_STATE_FAILED\x10\x05\x12\x1d\n\x19OPERATION_STATE_CANCELLED\x10\x06*Q\n\x10\x43ompressionCodec\x12!\n\x1d\x43OMPRESSION_CODEC_UNSPECIFIED\x10\x00\x12\x1a\n\x16\x43OMPRESSION_CODEC_ZSTD\x10\x01\x32\xdf\x08\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\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x12Y\n\x0c\x43loneSession\x12".spark.connect.CloneSessionRequest\x1a#.spark.connect.CloneSessionResponse"\x00\x12P\n\tGetStatus\x12\x1f.spark.connect.GetStatusRequest\x1a .spark.connect.GetStatusResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -71,8 +71,8 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_COMPRESSIONCODEC"]._serialized_start = 19991 - _globals["_COMPRESSIONCODEC"]._serialized_end = 20072 + _globals["_COMPRESSIONCODEC"]._serialized_start = 20312 + _globals["_COMPRESSIONCODEC"]._serialized_end = 20393 _globals["_PLAN"]._serialized_start = 275 _globals["_PLAN"]._serialized_end = 758 _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_start = 477 @@ -82,205 +82,209 @@ _globals["_USERCONTEXT"]._serialized_start = 760 _globals["_USERCONTEXT"]._serialized_end = 882 _globals["_ANALYZEPLANREQUEST"]._serialized_start = 885 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3562 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 2247 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 2296 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 2299 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2614 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2442 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2614 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2616 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2706 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2708 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2758 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2760 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2814 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2816 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2869 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2871 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2885 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2887 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2928 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2930 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 3051 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 3053 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 3108 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 3111 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 3262 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 3264 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3374 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3376 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3446 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3448 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3492 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3565 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5431 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4806 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4863 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4865 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4913 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4915 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4960 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4962 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4998 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 5000 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 5048 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 5050 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 5084 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 5086 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 5126 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 5128 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 5187 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 5189 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 5228 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 5230 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 5268 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 3111 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 3120 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 3264 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 3275 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 5294 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5377 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5379 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5421 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5434 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 6205 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5868 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 6129 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 6208 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 9799 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 8308 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8379 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8382 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8630 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8633 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 9150 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8728 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 9060 + _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3723 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 2347 + _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 2396 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 2399 + _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2714 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2542 + _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2714 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2716 + _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2806 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2808 + _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2858 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2860 + _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2914 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2916 + _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2969 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2971 + _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2985 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2987 + _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 3028 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 3030 + _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 3151 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 3153 + _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 3208 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 3211 + _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 3362 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 3364 + _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3474 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3476 + _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3546 + _globals["_ANALYZEPLANREQUEST_GETNUMPARTITIONS"]._serialized_start = 3548 + _globals["_ANALYZEPLANREQUEST_GETNUMPARTITIONS"]._serialized_end = 3607 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3609 + _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3653 + _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3726 + _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5752 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 5068 + _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 5125 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 5127 + _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 5175 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 5177 + _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 5222 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 5224 + _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 5260 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 5262 + _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 5310 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 5312 + _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 5346 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 5348 + _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 5388 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 5390 + _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 5449 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 5451 + _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 5490 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 5492 + _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 5530 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 3211 + _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 3220 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 3364 + _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 3375 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 5556 + _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5639 + _globals["_ANALYZEPLANRESPONSE_GETNUMPARTITIONS"]._serialized_start = 5641 + _globals["_ANALYZEPLANRESPONSE_GETNUMPARTITIONS"]._serialized_end = 5698 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5700 + _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5742 + _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5755 + _globals["_EXECUTEPLANREQUEST"]._serialized_end = 6526 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 6189 + _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 6450 + _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 6529 + _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 10120 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 8629 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8700 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8703 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8951 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8954 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 9471 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 9049 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 9381 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 8937 + ]._serialized_start = 9258 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 9060 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 9062 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 9150 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 9153 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 9428 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 9430 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 9446 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 9449 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9782 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9605 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9782 - _globals["_KEYVALUE"]._serialized_start = 9801 - _globals["_KEYVALUE"]._serialized_end = 9866 - _globals["_CONFIGREQUEST"]._serialized_start = 9869 - _globals["_CONFIGREQUEST"]._serialized_end = 11068 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 10177 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10675 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 10677 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 10769 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 10771 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 10796 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10798 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10861 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10863 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10894 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10896 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10944 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10946 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10973 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10975 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 11009 - _globals["_CONFIGRESPONSE"]._serialized_start = 11071 - _globals["_CONFIGRESPONSE"]._serialized_end = 11246 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 11249 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 12251 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11724 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11777 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11779 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11890 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11892 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11985 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11988 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 12181 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 12254 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12526 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12445 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12526 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12529 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12855 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12858 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 13210 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 13053 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 13168 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 13170 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 13210 - _globals["_INTERRUPTREQUEST"]._serialized_start = 13213 - _globals["_INTERRUPTREQUEST"]._serialized_end = 13816 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13616 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13744 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 13819 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 13963 - _globals["_REATTACHOPTIONS"]._serialized_start = 13965 - _globals["_REATTACHOPTIONS"]._serialized_end = 14018 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 14021 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 14202 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 14205 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14611 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14614 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 15199 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 15068 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 15080 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 15082 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 15129 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 15202 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 15367 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 15370 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15582 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15584 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15692 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15695 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 16027 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 16030 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 18039 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 16259 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16433 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16436 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16804 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16767 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16804 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16807 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 17357 + ]._serialized_end = 9381 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 9383 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 9471 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 9474 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 9749 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 9751 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 9767 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 9770 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 10103 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9926 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 10103 + _globals["_KEYVALUE"]._serialized_start = 10122 + _globals["_KEYVALUE"]._serialized_end = 10187 + _globals["_CONFIGREQUEST"]._serialized_start = 10190 + _globals["_CONFIGREQUEST"]._serialized_end = 11389 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 10498 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10996 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 10998 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 11090 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 11092 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 11117 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 11119 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 11182 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 11184 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 11215 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 11217 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 11265 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 11267 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 11294 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 11296 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 11330 + _globals["_CONFIGRESPONSE"]._serialized_start = 11392 + _globals["_CONFIGRESPONSE"]._serialized_end = 11567 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 11570 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 12572 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 12045 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 12098 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 12100 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 12211 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 12213 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 12306 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 12309 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 12502 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 12575 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12847 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12766 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12847 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12850 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 13176 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 13179 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 13531 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 13374 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 13489 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 13491 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 13531 + _globals["_INTERRUPTREQUEST"]._serialized_start = 13534 + _globals["_INTERRUPTREQUEST"]._serialized_end = 14137 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13937 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 14065 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 14140 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 14284 + _globals["_REATTACHOPTIONS"]._serialized_start = 14286 + _globals["_REATTACHOPTIONS"]._serialized_end = 14339 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 14342 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 14523 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 14526 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14932 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14935 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 15520 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 15389 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 15401 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 15403 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 15450 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 15523 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 15688 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 15691 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15903 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15905 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 16013 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 16016 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 16348 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 16351 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 18360 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 16580 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16754 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16757 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 17125 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 17088 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 17125 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 17128 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 17678 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 17234 + ]._serialized_start = 17555 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 17302 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 17360 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17610 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17612 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17670 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17673 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 18020 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 18041 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 18131 - _globals["_CLONESESSIONREQUEST"]._serialized_start = 18134 - _globals["_CLONESESSIONREQUEST"]._serialized_end = 18496 - _globals["_CLONESESSIONRESPONSE"]._serialized_start = 18499 - _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18703 - _globals["_GETSTATUSREQUEST"]._serialized_start = 18706 - _globals["_GETSTATUSREQUEST"]._serialized_end = 19301 - _globals["_GETSTATUSREQUEST_OPERATIONSTATUSREQUEST"]._serialized_start = 19105 - _globals["_GETSTATUSREQUEST_OPERATIONSTATUSREQUEST"]._serialized_end = 19221 - _globals["_GETSTATUSRESPONSE"]._serialized_start = 19304 - _globals["_GETSTATUSRESPONSE"]._serialized_end = 19989 - _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS"]._serialized_start = 19562 - _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS"]._serialized_end = 19989 - _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS_OPERATIONSTATE"]._serialized_start = 19759 - _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS_OPERATIONSTATE"]._serialized_end = 19989 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 20075 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 21194 + ]._serialized_end = 17623 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 17681 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17931 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17933 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17991 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17994 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 18341 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 18362 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 18452 + _globals["_CLONESESSIONREQUEST"]._serialized_start = 18455 + _globals["_CLONESESSIONREQUEST"]._serialized_end = 18817 + _globals["_CLONESESSIONRESPONSE"]._serialized_start = 18820 + _globals["_CLONESESSIONRESPONSE"]._serialized_end = 19024 + _globals["_GETSTATUSREQUEST"]._serialized_start = 19027 + _globals["_GETSTATUSREQUEST"]._serialized_end = 19622 + _globals["_GETSTATUSREQUEST_OPERATIONSTATUSREQUEST"]._serialized_start = 19426 + _globals["_GETSTATUSREQUEST_OPERATIONSTATUSREQUEST"]._serialized_end = 19542 + _globals["_GETSTATUSRESPONSE"]._serialized_start = 19625 + _globals["_GETSTATUSRESPONSE"]._serialized_end = 20310 + _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS"]._serialized_start = 19883 + _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS"]._serialized_end = 20310 + _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS_OPERATIONSTATE"]._serialized_start = 20080 + _globals["_GETSTATUSRESPONSE_OPERATIONSTATUS_OPERATIONSTATE"]._serialized_end = 20310 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 20396 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 21515 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 2db3132cd0c01..c5fea78671bbd 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -576,6 +576,23 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["json_string", b"json_string"] ) -> None: ... + class GetNumPartitions(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PLAN_FIELD_NUMBER: builtins.int + @property + def plan(self) -> global___Plan: + """(Required) The logical plan to be analyzed.""" + def __init__( + self, + *, + plan: global___Plan | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["plan", b"plan"] + ) -> builtins.bool: ... + def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int CLIENT_OBSERVED_SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int @@ -594,6 +611,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int JSON_TO_DDL_FIELD_NUMBER: builtins.int + GET_NUM_PARTITIONS_FIELD_NUMBER: builtins.int session_id: builtins.str """(Required) @@ -644,6 +662,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): def get_storage_level(self) -> global___AnalyzePlanRequest.GetStorageLevel: ... @property def json_to_ddl(self) -> global___AnalyzePlanRequest.JsonToDDL: ... + @property + def get_num_partitions(self) -> global___AnalyzePlanRequest.GetNumPartitions: ... def __init__( self, *, @@ -665,6 +685,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): unpersist: global___AnalyzePlanRequest.Unpersist | None = ..., get_storage_level: global___AnalyzePlanRequest.GetStorageLevel | None = ..., json_to_ddl: global___AnalyzePlanRequest.JsonToDDL | None = ..., + get_num_partitions: global___AnalyzePlanRequest.GetNumPartitions | None = ..., ) -> None: ... def HasField( self, @@ -683,6 +704,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"ddl_parse", "explain", b"explain", + "get_num_partitions", + b"get_num_partitions", "get_storage_level", b"get_storage_level", "input_files", @@ -728,6 +751,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"ddl_parse", "explain", b"explain", + "get_num_partitions", + b"get_num_partitions", "get_storage_level", b"get_storage_level", "input_files", @@ -788,6 +813,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): "unpersist", "get_storage_level", "json_to_ddl", + "get_num_partitions", ] | None ): ... @@ -999,6 +1025,21 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... + class GetNumPartitions(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + NUM_PARTITIONS_FIELD_NUMBER: builtins.int + num_partitions: builtins.int + """The number of partitions in the physical execution plan.""" + def __init__( + self, + *, + num_partitions: builtins.int = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["num_partitions", b"num_partitions"] + ) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int SCHEMA_FIELD_NUMBER: builtins.int @@ -1015,6 +1056,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int JSON_TO_DDL_FIELD_NUMBER: builtins.int + GET_NUM_PARTITIONS_FIELD_NUMBER: builtins.int session_id: builtins.str server_side_session_id: builtins.str """Server-side generated idempotency key that the client can use to assert that the server side @@ -1048,6 +1090,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): def get_storage_level(self) -> global___AnalyzePlanResponse.GetStorageLevel: ... @property def json_to_ddl(self) -> global___AnalyzePlanResponse.JsonToDDL: ... + @property + def get_num_partitions(self) -> global___AnalyzePlanResponse.GetNumPartitions: ... def __init__( self, *, @@ -1067,6 +1111,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): unpersist: global___AnalyzePlanResponse.Unpersist | None = ..., get_storage_level: global___AnalyzePlanResponse.GetStorageLevel | None = ..., json_to_ddl: global___AnalyzePlanResponse.JsonToDDL | None = ..., + get_num_partitions: global___AnalyzePlanResponse.GetNumPartitions | None = ..., ) -> None: ... def HasField( self, @@ -1075,6 +1120,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): b"ddl_parse", "explain", b"explain", + "get_num_partitions", + b"get_num_partitions", "get_storage_level", b"get_storage_level", "input_files", @@ -1110,6 +1157,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): b"ddl_parse", "explain", b"explain", + "get_num_partitions", + b"get_num_partitions", "get_storage_level", b"get_storage_level", "input_files", @@ -1160,6 +1209,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): "unpersist", "get_storage_level", "json_to_ddl", + "get_num_partitions", ] | None ): ... From e37527a8909177f1f5454713a170957a2d7f6e18 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 17:58:27 -0300 Subject: [PATCH 10/12] Fixing proto --- python/pyspark/sql/connect/proto/base_pb2.pyi | 54 +++++++++---------- 1 file changed, 27 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index c5fea78671bbd..8e58c12246b2b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -561,21 +561,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... - class JsonToDDL(google.protobuf.message.Message): - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - JSON_STRING_FIELD_NUMBER: builtins.int - json_string: builtins.str - """(Required) The JSON formatted string to be converted to DDL.""" - def __init__( - self, - *, - json_string: builtins.str = ..., - ) -> None: ... - def ClearField( - self, field_name: typing_extensions.Literal["json_string", b"json_string"] - ) -> None: ... - class GetNumPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -593,6 +578,21 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + class JsonToDDL(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + JSON_STRING_FIELD_NUMBER: builtins.int + json_string: builtins.str + """(Required) The JSON formatted string to be converted to DDL.""" + def __init__( + self, + *, + json_string: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["json_string", b"json_string"] + ) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int CLIENT_OBSERVED_SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int @@ -823,7 +823,7 @@ global___AnalyzePlanRequest = AnalyzePlanRequest class AnalyzePlanResponse(google.protobuf.message.Message): """Response to performing analysis of the query. Contains relevant metadata to be able to reason about the performance. - Next ID: 16 + Next ID: 18 """ DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1011,33 +1011,33 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["storage_level", b"storage_level"] ) -> None: ... - class JsonToDDL(google.protobuf.message.Message): + class GetNumPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - DDL_STRING_FIELD_NUMBER: builtins.int - ddl_string: builtins.str + NUM_PARTITIONS_FIELD_NUMBER: builtins.int + num_partitions: builtins.int + """The number of partitions in the physical execution plan.""" def __init__( self, *, - ddl_string: builtins.str = ..., + num_partitions: builtins.int = ..., ) -> None: ... def ClearField( - self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] + self, field_name: typing_extensions.Literal["num_partitions", b"num_partitions"] ) -> None: ... - class GetNumPartitions(google.protobuf.message.Message): + class JsonToDDL(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - NUM_PARTITIONS_FIELD_NUMBER: builtins.int - num_partitions: builtins.int - """The number of partitions in the physical execution plan.""" + DDL_STRING_FIELD_NUMBER: builtins.int + ddl_string: builtins.str def __init__( self, *, - num_partitions: builtins.int = ..., + ddl_string: builtins.str = ..., ) -> None: ... def ClearField( - self, field_name: typing_extensions.Literal["num_partitions", b"num_partitions"] + self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... SESSION_ID_FIELD_NUMBER: builtins.int From 4a90d85028f96cdb10e93b34d8fe2d9e50d4753f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Tue, 5 May 2026 19:35:04 -0300 Subject: [PATCH 11/12] Fixing compatibility check --- python/pyspark/sql/tests/test_connect_compatibility.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index bd49b1f465482..2d2ca62d5cd90 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -230,7 +230,7 @@ def test_dataframe_compatibility(self): expected_missing_connect_properties = {"sql_ctx"} expected_missing_classic_properties = {"is_cached"} expected_missing_connect_methods = set() - expected_missing_classic_methods = set() + expected_missing_classic_methods = {"getNumPartitions"} self.check_compatibility( ClassicDataFrame, ConnectDataFrame, From 127564162d2ec9206f9d7ecd44b9930fa2d31399 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Andr=C3=A9=20Fonseca?= <1077309+andreAmorimF@users.noreply.github.com> Date: Wed, 6 May 2026 10:38:40 -0300 Subject: [PATCH 12/12] Trigger Build