From b4b689028a5a28818bdeb5e1769ffd39d3fec3be Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Thu, 2 Oct 2025 12:43:29 +0000 Subject: [PATCH 01/19] Spark Connect Proto Plan Compression --- .../resources/error/error-conditions.json | 18 + python/packaging/classic/setup.py | 3 + python/packaging/client/setup.py | 2 + python/packaging/connect/setup.py | 2 + python/pyspark/sql/connect/client/core.py | 103 ++++- python/pyspark/sql/connect/plan.py | 3 +- python/pyspark/sql/connect/proto/base_pb2.py | 376 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 93 ++++- python/pyspark/sql/connect/utils.py | 17 + .../sql/tests/connect/test_connect_basic.py | 26 ++ .../sql/tests/connect/test_connect_plan.py | 2 +- python/pyspark/testing/connectutils.py | 13 + .../main/protobuf/spark/connect/base.proto | 26 +- .../spark/sql/connect/config/Connect.scala | 30 ++ .../service/SparkConnectAnalyzeHandler.scala | 22 +- .../SparkConnectExecutePlanHandler.scala | 11 +- .../service/SparkConnectServiceE2ESuite.scala | 180 +++++++++ 17 files changed, 717 insertions(+), 210 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 8f6687587f78..81c948731016 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -870,6 +870,24 @@ }, "sqlState" : "56K00" }, + "CONNECT_INVALID_PLAN" : { + "message" : [ + "The Spark Connect plan is invalid." + ], + "subClass" : { + "CANNOT_PARSE" : { + "message" : [ + "Cannot decompress or parse the input plan ()" + ] + }, + "PLAN_SIZE_LARGER_THAN_MAX" : { + "message" : [ + "The plan size is larger than max ( vs. )" + ] + } + }, + "sqlState" : "56K00" + }, "CONNECT_ML" : { "message" : [ "Generic Spark Connect ML error." diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index b2e4f2a4db55..5a5ada023116 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -156,6 +156,7 @@ def _supports_symlinks(): _minimum_grpc_version = "1.67.0" _minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" +_minimum_zstandard_version = "0.23.0" class InstallCommand(install): @@ -366,6 +367,7 @@ def run(self): "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, ], "pipelines": [ @@ -375,6 +377,7 @@ def run(self): "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], }, diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index 7ec7e45a3160..7652b7d28af2 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -139,6 +139,7 @@ _minimum_grpc_version = "1.67.0" _minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" + _minimum_zstandard_version = "0.23.0" with open("README.md") as f: long_description = f.read() @@ -211,6 +212,7 @@ "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index f2b53211b3a0..2fb3257bf6b9 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -92,6 +92,7 @@ _minimum_grpc_version = "1.67.0" _minimum_googleapis_common_protos_version = "1.65.0" _minimum_pyyaml_version = "3.11" + _minimum_zstandard_version = "0.23.0" with open("README.md") as f: long_description = f.read() @@ -121,6 +122,7 @@ "grpcio>=%s" % _minimum_grpc_version, "grpcio-status>=%s" % _minimum_grpc_version, "googleapis-common-protos>=%s" % _minimum_googleapis_common_protos_version, + "zstandard>=%s" % _minimum_zstandard_version, "numpy>=%s" % _minimum_numpy_version, "pyyaml>=%s" % _minimum_pyyaml_version, ], diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 741d612f53f4..4da73ff459a1 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -34,6 +34,7 @@ import urllib.parse import uuid import sys +import time from typing import ( Iterable, Iterator, @@ -113,6 +114,18 @@ from pyspark.sql.datasource import DataSource +def _import_zstandard_if_available(): + """ + Import zstandard if available, otherwise return None. + This is used to handle the case when zstandard is not installed. + """ + try: + import zstandard + return zstandard + except ImportError: + return None + + class ChannelBuilder: """ This is a helper class that is used to create a GRPC channel based on the given @@ -706,6 +719,10 @@ def __init__( self._progress_handlers: List[ProgressHandler] = [] + self._zstd_module = _import_zstandard_if_available() + self._plan_compression_threshold = None # Will be fetched lazily + self._plan_compression_algorithm = None # Will be fetched lazily + # cleanup ml cache if possible atexit.register(self._cleanup_ml_cache) @@ -1156,7 +1173,7 @@ def execute_command( req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id - req.plan.command.CopyFrom(command) + self._set_command_in_plan(req.plan, command) data, _, metrics, observed_metrics, properties = self._execute_and_fetch( req, observations or {} ) @@ -1182,7 +1199,7 @@ def execute_command_as_iterator( req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id - req.plan.command.CopyFrom(command) + self._set_command_in_plan(req.plan, command) for response in self._execute_and_fetch_as_iterator(req, observations or {}): if isinstance(response, dict): yield response @@ -1980,6 +1997,13 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: message=str(error), grpc_status_code=status_code, ) from None +# <<<<<<< +# ======= +# if info.metadata["errorClass"] == "CONNECT_INVALID_PLAN.CANNOT_PARSE": +# # Disable plan compression if the server fails to interpret the plan. +# logger.info(f"Disabling plan compression for the session due to {info.metadata["errorClass"]} error.") +# self._plan_compression_threshold, self._plan_compression_algorithm = -1, "NONE" +# >>>>>>> def add_artifacts(self, *paths: str, pyfile: bool, archive: bool, file: bool) -> None: try: @@ -2105,3 +2129,78 @@ def _query_model_size(self, model_ref_id: str) -> int: ml_command_result = properties["ml_command_result"] return ml_command_result.param.long + + def _set_relation_in_plan(self, plan: pb2.Plan, relation: pb2.Relation) -> None: + """Sets the relation in the plan, attempting compression if configured.""" + self._try_compress_and_set_plan( + plan=plan, + message=relation, + op_type=pb2.Plan.CompressedOperation.OpType.OP_TYPE_RELATION, + ) + + def _set_command_in_plan(self, plan: pb2.Plan, command: pb2.Command) -> None: + """Sets the command in the plan, attempting compression if configured.""" + self._try_compress_and_set_plan( + plan=plan, + message=command, + op_type=pb2.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND, + ) + + def _try_compress_and_set_plan( + self, + plan: pb2.Plan, + message: google.protobuf.message.Message, + op_type: pb2.Plan.CompressedOperation.OpType, + ) -> None: + """ + Tries to compress a protobuf message and sets it on the plan. + If compression is not enabled, not effective, or not available, + it falls back to the original message. + """ + plan_compression_threshold, plan_compression_algorithm = self._get_plan_compression_threshold_and_algorithm() + plan_compression_enabled = ( + plan_compression_threshold is not None and + plan_compression_threshold >= 0 and + plan_compression_algorithm is not None and + plan_compression_algorithm != "NONE" + ) + if plan_compression_enabled: + serialized_msg = message.SerializeToString() + original_size = len(serialized_msg) + if original_size > plan_compression_threshold and plan_compression_algorithm == "ZSTD" and self._zstd_module: + start_time = time.time() + compressed_operation = pb2.Plan.CompressedOperation( + data=self._zstd_module.compress(serialized_msg), + op_type=op_type, + compression_codec=pb2.CompressionCodec.COMPRESSION_CODEC_ZSTD, + ) + duration = time.time() - start_time + compressed_size = len(compressed_operation.data) + logger.debug(f"Plan compression: original_size={original_size}, compressed_size={compressed_size}, " + f"saving_ratio={1 - compressed_size / original_size:.2f}, duration_s={duration:.1f}") + if compressed_size < original_size: + plan.compressed_operation.CopyFrom(compressed_operation) + return + else: + logger.debug("Plan compression not effective. Using original plan.") + + if op_type == pb2.Plan.CompressedOperation.OpType.OP_TYPE_RELATION: + plan.root.CopyFrom(message) + else: + plan.command.CopyFrom(message) + + def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: + if self._plan_compression_threshold is None or self._plan_compression_algorithm is None: + try: + self._plan_compression_threshold, self._plan_compression_algorithm = self.get_configs( + "spark.connect.session.planCompression.threshold", + "spark.connect.session.planCompression.defaultAlgorithm", + ) + self._plan_compression_threshold = int(self._plan_compression_threshold) + logger.debug(f"Plan compression threshold: {self._plan_compression_threshold}, " + f"algorithm: {self._plan_compression_algorithm}") + except Exception as e: + self._plan_compression_threshold = -1 + self._plan_compression_algorithm = "NONE" + logger.debug("Plan compression is disabled because the server does not support it.", e) + return self._plan_compression_threshold, self._plan_compression_algorithm diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index c5b6f5430d6d..4359197831fc 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -143,7 +143,8 @@ def to_proto(self, session: "SparkConnectClient", debug: bool = False) -> proto. if enabled, the proto plan will be printed. """ plan = proto.Plan() - plan.root.CopyFrom(self.plan(session)) + relation = self.plan(session) + session._set_relation_in_plan(plan, relation) if debug: print(plan) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 269f5eefbd26..52cd4c5e2f23 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -45,7 +45,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"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"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"\x81\x1b\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\x8d\x01\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\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\x08relation2\xb2\x07\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\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"\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"\x81\x1b\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\x8d\x01\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\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*Q\n\x10\x43ompressionCodec\x12!\n\x1d\x43OMPRESSION_CODEC_UNSPECIFIED\x10\x00\x12\x1a\n\x16\x43OMPRESSION_CODEC_ZSTD\x10\x01\x32\xb2\x07\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\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -70,196 +70,202 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_PLAN"]._serialized_start = 274 - _globals["_PLAN"]._serialized_end = 390 - _globals["_USERCONTEXT"]._serialized_start = 392 - _globals["_USERCONTEXT"]._serialized_end = 514 - _globals["_ANALYZEPLANREQUEST"]._serialized_start = 517 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3194 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1879 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1928 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1931 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2074 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2248 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2338 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2340 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2390 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2392 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2446 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2448 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2501 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2503 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2517 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2519 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2560 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2562 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2683 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2685 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2740 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2894 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3006 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3008 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3078 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3080 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3124 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3197 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5063 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4438 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4495 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4497 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4545 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4547 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4592 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4594 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4630 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4632 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4680 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4682 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4716 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4718 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4758 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4760 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4819 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4821 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4860 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4862 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4900 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2752 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2907 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4926 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5009 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5011 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5053 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5066 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5837 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5500 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5761 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5840 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 9297 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7940 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8011 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8014 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8262 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8265 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8360 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8692 + _globals["_COMPRESSIONCODEC"]._serialized_start = 17999 + _globals["_COMPRESSIONCODEC"]._serialized_end = 18080 + _globals["_PLAN"]._serialized_start = 275 + _globals["_PLAN"]._serialized_end = 758 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_start = 477 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_end = 747 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_start = 671 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_end = 747 + _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 = 9665 + _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[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 8569 + ]._serialized_start = 8937 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 8692 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8694 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8785 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8926 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8928 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8944 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8947 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9280 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9103 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9280 - _globals["_KEYVALUE"]._serialized_start = 9299 - _globals["_KEYVALUE"]._serialized_end = 9364 - _globals["_CONFIGREQUEST"]._serialized_start = 9367 - _globals["_CONFIGREQUEST"]._serialized_end = 10566 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9675 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10173 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 10175 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 10267 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 10269 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 10294 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10296 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10359 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10361 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10392 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10394 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10442 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10444 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10471 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10473 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10507 - _globals["_CONFIGRESPONSE"]._serialized_start = 10569 - _globals["_CONFIGRESPONSE"]._serialized_end = 10744 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10747 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11749 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11222 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11275 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11277 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11388 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11390 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11483 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11486 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11679 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11752 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12024 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11943 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12024 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12027 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12353 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12356 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12708 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12551 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12666 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12668 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12708 - _globals["_INTERRUPTREQUEST"]._serialized_start = 12711 - _globals["_INTERRUPTREQUEST"]._serialized_end = 13314 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13114 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13242 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 13317 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 13461 - _globals["_REATTACHOPTIONS"]._serialized_start = 13463 - _globals["_REATTACHOPTIONS"]._serialized_end = 13516 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13519 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 13700 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 13703 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14109 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14112 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 14697 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14566 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14578 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14580 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14627 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 14700 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14865 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14868 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15080 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15082 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15190 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15193 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15525 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15528 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17537 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 15757 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15931 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15934 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16265 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16305 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 16855 + ]._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 = 9294 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 9296 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 9312 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 9315 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9648 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9471 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9648 + _globals["_KEYVALUE"]._serialized_start = 9667 + _globals["_KEYVALUE"]._serialized_end = 9732 + _globals["_CONFIGREQUEST"]._serialized_start = 9735 + _globals["_CONFIGREQUEST"]._serialized_end = 10934 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 10043 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10541 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 10543 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 10635 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 10637 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 10662 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10664 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10727 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10729 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10760 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10762 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10810 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10812 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10839 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10841 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10875 + _globals["_CONFIGRESPONSE"]._serialized_start = 10937 + _globals["_CONFIGRESPONSE"]._serialized_end = 11112 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 11115 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 12117 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11590 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11643 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11645 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11756 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11758 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11851 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11854 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 12047 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 12120 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12392 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12311 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12392 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12395 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12721 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12724 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 13076 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12919 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 13034 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 13036 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 13076 + _globals["_INTERRUPTREQUEST"]._serialized_start = 13079 + _globals["_INTERRUPTREQUEST"]._serialized_end = 13682 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13482 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13610 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 13685 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 13829 + _globals["_REATTACHOPTIONS"]._serialized_start = 13831 + _globals["_REATTACHOPTIONS"]._serialized_end = 13884 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13887 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 14068 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 14071 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14477 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14480 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 15065 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14934 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14946 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14948 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14995 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 15068 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 15233 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 15236 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15448 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15450 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15558 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15561 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15893 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15896 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17905 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 16125 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16299 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16302 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16633 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16673 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 17223 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 16732 + ]._serialized_start = 17100 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 16800 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 16858 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17108 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17110 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17168 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17171 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17518 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17539 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17629 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 17632 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 18578 + ]._serialized_end = 17168 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 17226 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17476 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17478 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17536 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17539 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17886 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17907 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17997 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 18083 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 19029 # @@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 3361649d9323..50c05103009b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -57,42 +57,123 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +class _CompressionCodec: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + +class _CompressionCodecEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[_CompressionCodec.ValueType], + builtins.type, +): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + COMPRESSION_CODEC_UNSPECIFIED: _CompressionCodec.ValueType # 0 + COMPRESSION_CODEC_ZSTD: _CompressionCodec.ValueType # 1 + +class CompressionCodec(_CompressionCodec, metaclass=_CompressionCodecEnumTypeWrapper): + """Compression codec for plan compression.""" + +COMPRESSION_CODEC_UNSPECIFIED: CompressionCodec.ValueType # 0 +COMPRESSION_CODEC_ZSTD: CompressionCodec.ValueType # 1 +global___CompressionCodec = CompressionCodec + class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the - client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference - to the underlying logical plan or it can be of the [[Command]] type that is used to execute - commands on the server. + client to the server. A [[Plan]] can be one of the following: + - [[Relation]]: a reference to the underlying logical plan. + - [[Command]]: used to execute commands on the server. + - [[CompressedOperation]]: a compressed representation of either a Relation or a Command. """ DESCRIPTOR: google.protobuf.descriptor.Descriptor + class CompressedOperation(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _OpType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _OpTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + Plan.CompressedOperation._OpType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + OP_TYPE_UNSPECIFIED: Plan.CompressedOperation._OpType.ValueType # 0 + OP_TYPE_RELATION: Plan.CompressedOperation._OpType.ValueType # 1 + OP_TYPE_COMMAND: Plan.CompressedOperation._OpType.ValueType # 2 + + class OpType(_OpType, metaclass=_OpTypeEnumTypeWrapper): ... + OP_TYPE_UNSPECIFIED: Plan.CompressedOperation.OpType.ValueType # 0 + OP_TYPE_RELATION: Plan.CompressedOperation.OpType.ValueType # 1 + OP_TYPE_COMMAND: Plan.CompressedOperation.OpType.ValueType # 2 + + DATA_FIELD_NUMBER: builtins.int + OP_TYPE_FIELD_NUMBER: builtins.int + COMPRESSION_CODEC_FIELD_NUMBER: builtins.int + data: builtins.bytes + op_type: global___Plan.CompressedOperation.OpType.ValueType + compression_codec: global___CompressionCodec.ValueType + def __init__( + self, + *, + data: builtins.bytes = ..., + op_type: global___Plan.CompressedOperation.OpType.ValueType = ..., + compression_codec: global___CompressionCodec.ValueType = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "compression_codec", b"compression_codec", "data", b"data", "op_type", b"op_type" + ], + ) -> None: ... + ROOT_FIELD_NUMBER: builtins.int COMMAND_FIELD_NUMBER: builtins.int + COMPRESSED_OPERATION_FIELD_NUMBER: builtins.int @property def root(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: ... @property def command(self) -> pyspark.sql.connect.proto.commands_pb2.Command: ... + @property + def compressed_operation(self) -> global___Plan.CompressedOperation: ... def __init__( self, *, root: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., command: pyspark.sql.connect.proto.commands_pb2.Command | None = ..., + compressed_operation: global___Plan.CompressedOperation | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "command", b"command", "op_type", b"op_type", "root", b"root" + "command", + b"command", + "compressed_operation", + b"compressed_operation", + "op_type", + b"op_type", + "root", + b"root", ], ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "command", b"command", "op_type", b"op_type", "root", b"root" + "command", + b"command", + "compressed_operation", + b"compressed_operation", + "op_type", + b"op_type", + "root", + b"root", ], ) -> None: ... def WhichOneof( self, oneof_group: typing_extensions.Literal["op_type", b"op_type"] - ) -> typing_extensions.Literal["root", "command"] | None: ... + ) -> typing_extensions.Literal["root", "command", "compressed_operation"] | None: ... global___Plan = Plan diff --git a/python/pyspark/sql/connect/utils.py b/python/pyspark/sql/connect/utils.py index a2511836816c..bc29a1fd67bc 100644 --- a/python/pyspark/sql/connect/utils.py +++ b/python/pyspark/sql/connect/utils.py @@ -37,6 +37,7 @@ def check_dependencies(mod_name: str) -> None: require_minimum_grpc_version() require_minimum_grpcio_status_version() require_minimum_googleapis_common_protos_version() + require_minimum_zstandard_version() def require_minimum_grpc_version() -> None: @@ -96,5 +97,21 @@ def require_minimum_googleapis_common_protos_version() -> None: ) from error +def require_minimum_zstandard_version() -> None: + """Raise ImportError if zstandard is not installed""" + minimum_zstandard_version = "0.23.0" + + try: + import zstandard # noqa + except ImportError as error: + raise PySparkImportError( + errorClass="PACKAGE_NOT_INSTALLED", + messageParameters={ + "package_name": "zstandard", + "minimum_version": str(minimum_zstandard_version), + }, + ) from error + + def get_python_ver() -> str: return "%d.%d" % sys.version_info[:2] diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index c5b16f0da49e..87c2331ee242 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -1432,6 +1432,32 @@ def test_truncate_message(self): proto_string_truncated_3 = self.connect._client._proto_to_string(plan3, True) self.assertTrue(len(proto_string_truncated_3) < 64000, len(proto_string_truncated_3)) + def test_plan_compression(self): + self.assertTrue(self.connect._client._zstd_module is not None) + self.connect.range(1).count() + default_plan_compression_threshold = self.connect._client._plan_compression_threshold + self.assertTrue(default_plan_compression_threshold > 0) + self.assertTrue(self.connect._client._plan_compression_algorithm == "ZSTD") + try: + self.connect._client._plan_compression_threshold = 1000 + + # Small plan should not be compressed + cdf1 = self.connect.range(1).select(CF.lit("Apache Spark")) + plan1 = cdf1._plan.to_proto(self.connect._client) + self.assertTrue(plan1.root is not None) + self.assertTrue(cdf1.count() == 1) + + # Large plan should be compressed + cdf2 = self.connect.range(1).select(CF.lit("Apache Spark" * 1000)) + plan2 = cdf2._plan.to_proto(self.connect._client) + self.assertTrue(plan2.compressed_operation is not None) + # Test compressed relation + self.assertTrue(cdf2.count() == 1) + # Test compressed command + cdf2.createOrReplaceTempView("temp_view_cdf2") + self.assertTrue(self.connect.sql("SELECT * FROM temp_view_cdf2").count() == 1) + finally: + self.connect._client._plan_compression_threshold = default_plan_compression_threshold class SparkConnectGCTests(SparkConnectSQLTestCase): @classmethod diff --git a/python/pyspark/sql/tests/connect/test_connect_plan.py b/python/pyspark/sql/tests/connect/test_connect_plan.py index d25799f0c9f2..1d4d85e8426e 100644 --- a/python/pyspark/sql/tests/connect/test_connect_plan.py +++ b/python/pyspark/sql/tests/connect/test_connect_plan.py @@ -864,7 +864,7 @@ def test_uuid_literal(self): def test_column_literals(self): df = self.connect.with_plan(Read("table")) lit_df = df.select(lit(10)) - self.assertIsNotNone(lit_df._plan.to_proto(None)) + self.assertIsNotNone(lit_df._plan.to_proto(self.connect)) self.assertIsNotNone(lit(10).to_plan(None)) plan = lit(10).to_plan(None) diff --git a/python/pyspark/testing/connectutils.py b/python/pyspark/testing/connectutils.py index 1f5c5a086abf..d3a24ecf55b4 100644 --- a/python/pyspark/testing/connectutils.py +++ b/python/pyspark/testing/connectutils.py @@ -23,6 +23,7 @@ import uuid import contextlib +import pyspark.sql.connect.proto as pb2 from pyspark import Row, SparkConf from pyspark.util import is_remote_only from pyspark.testing.utils import PySparkErrorTestUtils @@ -113,6 +114,16 @@ def _session_range( def _session_sql(cls, query): return cls._df_mock(SQL(query)) + @classmethod + def _set_relation_in_plan(self, plan: pb2.Plan, relation: pb2.Relation) -> None: + # Skip plan compression in plan-only tests. + plan.root.CopyFrom(relation) + + @classmethod + def _set_command_in_plan(self, plan: pb2.Plan, command: pb2.Command) -> None: + # Skip plan compression in plan-only tests. + plan.command.CopyFrom(command) + if have_pandas: @classmethod @@ -129,6 +140,8 @@ def setUpClass(cls): cls.connect.set_hook("range", cls._session_range) cls.connect.set_hook("sql", cls._session_sql) cls.connect.set_hook("with_plan", cls._with_plan) + cls.connect.set_hook("_set_relation_in_plan", cls._set_relation_in_plan) + cls.connect.set_hook("_set_command_in_plan", cls._set_command_in_plan) @classmethod def tearDownClass(cls): 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 f34b169f821d..5168205b41e9 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -33,17 +33,35 @@ option java_package = "org.apache.spark.connect.proto"; option go_package = "internal/generated"; // A [[Plan]] is the structure that carries the runtime information for the execution from the -// client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference -// to the underlying logical plan or it can be of the [[Command]] type that is used to execute -// commands on the server. +// client to the server. A [[Plan]] can be one of the following: +// - [[Relation]]: a reference to the underlying logical plan. +// - [[Command]]: used to execute commands on the server. +// - [[CompressedOperation]]: a compressed representation of either a Relation or a Command. message Plan { oneof op_type { Relation root = 1; Command command = 2; + CompressedOperation compressed_operation = 3; } -} + message CompressedOperation { + bytes data = 1; + OpType op_type = 2; + CompressionCodec compression_codec = 3; + enum OpType { + OP_TYPE_UNSPECIFIED = 0; + OP_TYPE_RELATION = 1; + OP_TYPE_COMMAND = 2; + } + } +} + +// Compression codec for plan compression. +enum CompressionCodec { + COMPRESSION_CODEC_UNSPECIFIED = 0; + COMPRESSION_CODEC_ZSTD = 1; +} // User Context is used to refer to one particular user session that is executing // queries in the backend. diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index c6049187f6be..37f2c44a173e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.connect.config +import java.util.Locale import java.util.concurrent.TimeUnit import org.apache.spark.SparkEnv @@ -418,4 +419,33 @@ object Connect { .bytesConf(ByteUnit.BYTE) // 90% of the max message size by default to allow for some overhead. .createWithDefault((ConnectCommon.CONNECT_GRPC_MAX_MESSAGE_SIZE * 0.9).toInt) + + private[spark] val CONNECT_MAX_PLAN_SIZE = + buildStaticConf("spark.connect.maxPlanSize") + .doc("The maximum size of a (decompressed) proto plan that can be executed in Spark " + + "Connect. If the size of the plan exceeds this limit, an error will be thrown. " + + "The size is in bytes.") + .version("4.1.0") + .internal() + .bytesConf(ByteUnit.BYTE) + .createWithDefault(512 * 1024 * 1024) // 512 MB + + val CONNECT_SESSION_PLAN_COMPRESSION_THRESHOLD = + buildConf("spark.connect.session.planCompression.threshold") + .doc("The threshold in bytes for the size of proto plan to be compressed. " + + "If the size of proto plan is smaller than this threshold, it will not be compressed.") + .version("4.1.0") + .internal() + .intConf + .createWithDefault(10 * 1024 * 1024) // 10 MB + + val CONNECT_PLAN_COMPRESSION_DEFAULT_ALGORITHM = + buildConf("spark.connect.session.planCompression.defaultAlgorithm") + .doc("The default algorithm of proto plan compression.") + .version("4.1.0") + .internal() + .stringConf + .transform(_.toUpperCase(Locale.ROOT)) + .checkValues(ConnectPlanCompressionAlgorithm.values.map(_.toString)) + .createWithDefault(ConnectPlanCompressionAlgorithm.ZSTD.toString) } 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 8fa003c11681..cdf7013211f7 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 @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.classic.{DataFrame, Dataset} import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, StorageLevelProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.utils.PlanCompressionUtils import org.apache.spark.sql.execution.{CodegenMode, CommandExecutionMode, CostMode, ExtendedMode, FormattedMode, SimpleMode} import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.ArrayImplicits._ @@ -63,6 +64,9 @@ private[connect] class SparkConnectAnalyzeHandler( val builder = proto.AnalyzePlanResponse.newBuilder() def transformRelation(rel: proto.Relation) = planner.transformRelation(rel, cachePlan = true) + def transformRelationPlan(plan: proto.Plan) = { + transformRelation(PlanCompressionUtils.decompressPlan(plan).getRoot) + } def getDataFrameWithoutExecuting(rel: LogicalPlan): DataFrame = { val qe = session.sessionState.executePlan(rel, CommandExecutionMode.SKIP) @@ -71,7 +75,7 @@ private[connect] class SparkConnectAnalyzeHandler( request.getAnalyzeCase match { case proto.AnalyzePlanRequest.AnalyzeCase.SCHEMA => - val rel = transformRelation(request.getSchema.getPlan.getRoot) + val rel = transformRelationPlan(request.getSchema.getPlan) val schema = getDataFrameWithoutExecuting(rel).schema builder.setSchema( proto.AnalyzePlanResponse.Schema @@ -79,7 +83,7 @@ private[connect] class SparkConnectAnalyzeHandler( .setSchema(DataTypeProtoConverter.toConnectProtoType(schema)) .build()) case proto.AnalyzePlanRequest.AnalyzeCase.EXPLAIN => - val rel = transformRelation(request.getExplain.getPlan.getRoot) + val rel = transformRelationPlan(request.getExplain.getPlan) val queryExecution = getDataFrameWithoutExecuting(rel).queryExecution val explainString = request.getExplain.getExplainMode match { case proto.AnalyzePlanRequest.Explain.ExplainMode.EXPLAIN_MODE_SIMPLE => @@ -101,7 +105,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.TREE_STRING => - val rel = transformRelation(request.getTreeString.getPlan.getRoot) + val rel = transformRelationPlan(request.getTreeString.getPlan) val schema = getDataFrameWithoutExecuting(rel).schema val treeString = if (request.getTreeString.hasLevel) { schema.treeString(request.getTreeString.getLevel) @@ -115,7 +119,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.IS_LOCAL => - val rel = transformRelation(request.getIsLocal.getPlan.getRoot) + val rel = transformRelationPlan(request.getIsLocal.getPlan) val isLocal = getDataFrameWithoutExecuting(rel).isLocal builder.setIsLocal( proto.AnalyzePlanResponse.IsLocal @@ -124,7 +128,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.IS_STREAMING => - val rel = transformRelation(request.getIsStreaming.getPlan.getRoot) + val rel = transformRelationPlan(request.getIsStreaming.getPlan) val isStreaming = getDataFrameWithoutExecuting(rel).isStreaming builder.setIsStreaming( proto.AnalyzePlanResponse.IsStreaming @@ -133,7 +137,7 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.INPUT_FILES => - val rel = transformRelation(request.getInputFiles.getPlan.getRoot) + val rel = transformRelationPlan(request.getInputFiles.getPlan) val inputFiles = getDataFrameWithoutExecuting(rel).inputFiles builder.setInputFiles( proto.AnalyzePlanResponse.InputFiles @@ -157,8 +161,8 @@ private[connect] class SparkConnectAnalyzeHandler( .build()) case proto.AnalyzePlanRequest.AnalyzeCase.SAME_SEMANTICS => - val targetRel = transformRelation(request.getSameSemantics.getTargetPlan.getRoot) - val otherRel = transformRelation(request.getSameSemantics.getOtherPlan.getRoot) + val targetRel = transformRelationPlan(request.getSameSemantics.getTargetPlan) + val otherRel = transformRelationPlan(request.getSameSemantics.getOtherPlan) val target = getDataFrameWithoutExecuting(targetRel) val other = getDataFrameWithoutExecuting(otherRel) builder.setSameSemantics( @@ -167,7 +171,7 @@ private[connect] class SparkConnectAnalyzeHandler( .setResult(target.sameSemantics(other))) case proto.AnalyzePlanRequest.AnalyzeCase.SEMANTIC_HASH => - val rel = transformRelation(request.getSemanticHash.getPlan.getRoot) + val rel = transformRelationPlan(request.getSemanticHash.getPlan) val semanticHash = getDataFrameWithoutExecuting(rel) .semanticHash() builder.setSemanticHash( diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala index 027f4517cf3b..849479954de4 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala @@ -22,6 +22,7 @@ import io.grpc.stub.StreamObserver import org.apache.spark.SparkSQLException import org.apache.spark.connect.proto import org.apache.spark.internal.Logging +import org.apache.spark.sql.connect.utils.PlanCompressionUtils class SparkConnectExecutePlanHandler(responseObserver: StreamObserver[proto.ExecutePlanResponse]) extends Logging { @@ -35,12 +36,18 @@ class SparkConnectExecutePlanHandler(responseObserver: StreamObserver[proto.Exec .getOrCreateIsolatedSession(v.getUserContext.getUserId, v.getSessionId, previousSessionId) val executeKey = ExecuteKey(v, sessionHolder) + val decompressedRequest = + v.toBuilder.setPlan(PlanCompressionUtils.decompressPlan(v.getPlan)).build() + SparkConnectService.executionManager.getExecuteHolder(executeKey) match { case None => // Create a new execute holder and attach to it. SparkConnectService.executionManager - .createExecuteHolderAndAttach(executeKey, v, sessionHolder, responseObserver) - case Some(executeHolder) if executeHolder.request.getPlan.equals(v.getPlan) => + .createExecuteHolderAndAttach( + executeKey, decompressedRequest, sessionHolder, responseObserver + ) + case Some(executeHolder) + if executeHolder.request.getPlan.equals(decompressedRequest.getPlan) => // If the execute holder already exists with the same plan, reattach to it. SparkConnectService.executionManager .reattachExecuteHolder(executeHolder, responseObserver, None) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala index e3ba35073f41..db9e915e132f 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala @@ -16,12 +16,16 @@ */ package org.apache.spark.sql.connect.service +import java.io.ByteArrayOutputStream import java.util.UUID +import com.github.luben.zstd.{Zstd, ZstdOutputStreamNoFinalizer} +import com.google.protobuf.ByteString import org.scalatest.concurrent.Eventually import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.connect.proto import org.apache.spark.sql.connect.SparkConnectServerTest import org.apache.spark.sql.connect.config.Connect @@ -338,4 +342,180 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { assert(error.getMessage.contains(fixedOperationId)) } } + + test("Relation as compressed plan works") { + withClient { client => + val relation = buildPlan("SELECT 1").getRoot + val compressedRelation = Zstd.compress(relation.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + test("Command as compressed plan works") { + withClient { client => + val command = buildSqlCommandPlan("SET spark.sql.session.timeZone=Europe/Berlin").getCommand + val compressedCommand = Zstd.compress(command.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(compressedCommand)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + private def compressInZstdStreamingMode(input: Array[Byte]): Array[Byte] = { + val outputStream = new ByteArrayOutputStream() + val zstdStream = new ZstdOutputStreamNoFinalizer(outputStream) + zstdStream.write(input) + zstdStream.flush() + zstdStream.close() + outputStream.toByteArray + } + + test("Compressed plans generated in streaming mode also work correctly") { + withClient { client => + val relation = buildPlan("SELECT 1").getRoot + val compressedRelation = compressInZstdStreamingMode(relation.toByteArray) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val query = client.execute(plan) + while (query.hasNext) query.next() + } + } + + test("Invalid compressed bytes errors out") { + withClient { client => + val invalidBytes = "invalidBytes".getBytes + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(invalidBytes)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.CANNOT_PARSE")) + } + } + + test("Invalid compressed proto message errors out") { + withClient { client => + val data = Zstd.compress("Apache Spark".getBytes) + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(data)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.CANNOT_PARSE")) + } + } + + test("Large compressed plan errors out") { + withClient { client => + withSparkEnvConfs( + Connect.CONNECT_MAX_PLAN_SIZE.key -> "100" + ) { + val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot + val compressedRelation = Zstd.compress(relation.toByteArray) + + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX")) + } + } + } + + test("Large compressed plan generated in streaming mode also errors out") { + withClient { client => + withSparkEnvConfs( + Connect.CONNECT_MAX_PLAN_SIZE.key -> "100" + ) { + val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot + val compressedRelation = compressInZstdStreamingMode(relation.toByteArray) + + val plan = proto.Plan + .newBuilder() + .setCompressedOperation( + proto.Plan.CompressedOperation.newBuilder() + .setData(ByteString.copyFrom(compressedRelation)) + .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) + .setCompressionCodec( + proto.CompressionCodec.COMPRESSION_CODEC_ZSTD + ) + .build() + ) + .build() + val ex = intercept[SparkException] { + val query = client.execute(plan) + while (query.hasNext) query.next() + } + assert(ex.getMessage.contains("CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX")) + } + } + } } From f3fdaa095c5835aff37b53dd6337a2080b2416d5 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Thu, 2 Oct 2025 12:45:20 +0000 Subject: [PATCH 02/19] Reformat --- .../spark/sql/connect/config/Connect.scala | 7 +- .../SparkConnectExecutePlanHandler.scala | 8 +- .../service/SparkConnectServiceE2ESuite.scala | 78 +++++++------------ 3 files changed, 39 insertions(+), 54 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala index 37f2c44a173e..1ffed714b4ca 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala @@ -422,9 +422,10 @@ object Connect { private[spark] val CONNECT_MAX_PLAN_SIZE = buildStaticConf("spark.connect.maxPlanSize") - .doc("The maximum size of a (decompressed) proto plan that can be executed in Spark " + - "Connect. If the size of the plan exceeds this limit, an error will be thrown. " + - "The size is in bytes.") + .doc( + "The maximum size of a (decompressed) proto plan that can be executed in Spark " + + "Connect. If the size of the plan exceeds this limit, an error will be thrown. " + + "The size is in bytes.") .version("4.1.0") .internal() .bytesConf(ByteUnit.BYTE) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala index 849479954de4..6780ca37e96a 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala @@ -44,10 +44,12 @@ class SparkConnectExecutePlanHandler(responseObserver: StreamObserver[proto.Exec // Create a new execute holder and attach to it. SparkConnectService.executionManager .createExecuteHolderAndAttach( - executeKey, decompressedRequest, sessionHolder, responseObserver - ) + executeKey, + decompressedRequest, + sessionHolder, + responseObserver) case Some(executeHolder) - if executeHolder.request.getPlan.equals(decompressedRequest.getPlan) => + if executeHolder.request.getPlan.equals(decompressedRequest.getPlan) => // If the execute holder already exists with the same plan, reattach to it. SparkConnectService.executionManager .reattachExecuteHolder(executeHolder, responseObserver, None) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala index db9e915e132f..9136e1d42682 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala @@ -350,14 +350,12 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(compressedRelation)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val query = client.execute(plan) while (query.hasNext) query.next() @@ -371,14 +369,12 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(compressedCommand)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val query = client.execute(plan) while (query.hasNext) query.next() @@ -401,14 +397,12 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(compressedRelation)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val query = client.execute(plan) while (query.hasNext) query.next() @@ -421,14 +415,12 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(invalidBytes)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val ex = intercept[SparkException] { val query = client.execute(plan) @@ -444,14 +436,12 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(data)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val ex = intercept[SparkException] { val query = client.execute(plan) @@ -463,23 +453,19 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { test("Large compressed plan errors out") { withClient { client => - withSparkEnvConfs( - Connect.CONNECT_MAX_PLAN_SIZE.key -> "100" - ) { + withSparkEnvConfs(Connect.CONNECT_MAX_PLAN_SIZE.key -> "100") { val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot val compressedRelation = Zstd.compress(relation.toByteArray) val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(compressedRelation)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val ex = intercept[SparkException] { val query = client.execute(plan) @@ -492,23 +478,19 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { test("Large compressed plan generated in streaming mode also errors out") { withClient { client => - withSparkEnvConfs( - Connect.CONNECT_MAX_PLAN_SIZE.key -> "100" - ) { + withSparkEnvConfs(Connect.CONNECT_MAX_PLAN_SIZE.key -> "100") { val relation = buildPlan("SELECT '" + "Apache Spark" * 100 + "'").getRoot val compressedRelation = compressInZstdStreamingMode(relation.toByteArray) val plan = proto.Plan .newBuilder() .setCompressedOperation( - proto.Plan.CompressedOperation.newBuilder() + proto.Plan.CompressedOperation + .newBuilder() .setData(ByteString.copyFrom(compressedRelation)) .setOpType(proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION) - .setCompressionCodec( - proto.CompressionCodec.COMPRESSION_CODEC_ZSTD - ) - .build() - ) + .setCompressionCodec(proto.CompressionCodec.COMPRESSION_CODEC_ZSTD) + .build()) .build() val ex = intercept[SparkException] { val query = client.execute(plan) From 291c16dbb181f8ba3f611b03fdd2d3c585aa21b4 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Wed, 8 Oct 2025 12:32:59 +0000 Subject: [PATCH 03/19] Handle CONNECT_INVALID_PLAN.CANNOT_PARSE error --- python/pyspark/sql/connect/client/core.py | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 4da73ff459a1..d851ff26dde7 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -1980,6 +1980,11 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: if info.metadata.get("errorClass") == "INVALID_HANDLE.SESSION_CHANGED": self._closed = True + if info.metadata.get("errorClass") == "CONNECT_INVALID_PLAN.CANNOT_PARSE": + # Disable plan compression if the server fails to interpret the plan. + logger.info(f"Disabling plan compression for the session due to {info.metadata["errorClass"]} error.") + self._plan_compression_threshold, self._plan_compression_algorithm = -1, "NONE" + raise convert_exception( info, status.message, @@ -1997,13 +2002,6 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: message=str(error), grpc_status_code=status_code, ) from None -# <<<<<<< -# ======= -# if info.metadata["errorClass"] == "CONNECT_INVALID_PLAN.CANNOT_PARSE": -# # Disable plan compression if the server fails to interpret the plan. -# logger.info(f"Disabling plan compression for the session due to {info.metadata["errorClass"]} error.") -# self._plan_compression_threshold, self._plan_compression_algorithm = -1, "NONE" -# >>>>>>> def add_artifacts(self, *paths: str, pyfile: bool, archive: bool, file: bool) -> None: try: From b0c2e0785d79e99d7b34f6b5ca950a753758810a Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Wed, 8 Oct 2025 12:34:32 +0000 Subject: [PATCH 04/19] Reformat --- python/pyspark/sql/connect/client/core.py | 58 +++++++++++++------ .../sql/tests/connect/test_connect_basic.py | 1 + 2 files changed, 41 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index d851ff26dde7..85a0f4f22164 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -121,6 +121,7 @@ def _import_zstandard_if_available(): """ try: import zstandard + return zstandard except ImportError: return None @@ -1982,8 +1983,13 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: if info.metadata.get("errorClass") == "CONNECT_INVALID_PLAN.CANNOT_PARSE": # Disable plan compression if the server fails to interpret the plan. - logger.info(f"Disabling plan compression for the session due to {info.metadata["errorClass"]} error.") - self._plan_compression_threshold, self._plan_compression_algorithm = -1, "NONE" + logger.info( + "Disabling plan compression for the session due to CONNECT_INVALID_PLAN.CANNOT_PARSE error." + ) + self._plan_compression_threshold, self._plan_compression_algorithm = ( + -1, + "NONE", + ) raise convert_exception( info, @@ -2145,27 +2151,34 @@ def _set_command_in_plan(self, plan: pb2.Plan, command: pb2.Command) -> None: ) def _try_compress_and_set_plan( - self, - plan: pb2.Plan, - message: google.protobuf.message.Message, - op_type: pb2.Plan.CompressedOperation.OpType, + self, + plan: pb2.Plan, + message: google.protobuf.message.Message, + op_type: pb2.Plan.CompressedOperation.OpType, ) -> None: """ Tries to compress a protobuf message and sets it on the plan. If compression is not enabled, not effective, or not available, it falls back to the original message. """ - plan_compression_threshold, plan_compression_algorithm = self._get_plan_compression_threshold_and_algorithm() + ( + plan_compression_threshold, + plan_compression_algorithm, + ) = self._get_plan_compression_threshold_and_algorithm() plan_compression_enabled = ( - plan_compression_threshold is not None and - plan_compression_threshold >= 0 and - plan_compression_algorithm is not None and - plan_compression_algorithm != "NONE" + plan_compression_threshold is not None + and plan_compression_threshold >= 0 + and plan_compression_algorithm is not None + and plan_compression_algorithm != "NONE" ) if plan_compression_enabled: serialized_msg = message.SerializeToString() original_size = len(serialized_msg) - if original_size > plan_compression_threshold and plan_compression_algorithm == "ZSTD" and self._zstd_module: + if ( + original_size > plan_compression_threshold + and plan_compression_algorithm == "ZSTD" + and self._zstd_module + ): start_time = time.time() compressed_operation = pb2.Plan.CompressedOperation( data=self._zstd_module.compress(serialized_msg), @@ -2174,8 +2187,10 @@ def _try_compress_and_set_plan( ) duration = time.time() - start_time compressed_size = len(compressed_operation.data) - logger.debug(f"Plan compression: original_size={original_size}, compressed_size={compressed_size}, " - f"saving_ratio={1 - compressed_size / original_size:.2f}, duration_s={duration:.1f}") + logger.debug( + f"Plan compression: original_size={original_size}, compressed_size={compressed_size}, " + f"saving_ratio={1 - compressed_size / original_size:.2f}, duration_s={duration:.1f}" + ) if compressed_size < original_size: plan.compressed_operation.CopyFrom(compressed_operation) return @@ -2190,15 +2205,22 @@ def _try_compress_and_set_plan( def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: if self._plan_compression_threshold is None or self._plan_compression_algorithm is None: try: - self._plan_compression_threshold, self._plan_compression_algorithm = self.get_configs( + ( + self._plan_compression_threshold, + self._plan_compression_algorithm, + ) = self.get_configs( "spark.connect.session.planCompression.threshold", "spark.connect.session.planCompression.defaultAlgorithm", ) self._plan_compression_threshold = int(self._plan_compression_threshold) - logger.debug(f"Plan compression threshold: {self._plan_compression_threshold}, " - f"algorithm: {self._plan_compression_algorithm}") + logger.debug( + f"Plan compression threshold: {self._plan_compression_threshold}, " + f"algorithm: {self._plan_compression_algorithm}" + ) except Exception as e: self._plan_compression_threshold = -1 self._plan_compression_algorithm = "NONE" - logger.debug("Plan compression is disabled because the server does not support it.", e) + logger.debug( + "Plan compression is disabled because the server does not support it.", e + ) return self._plan_compression_threshold, self._plan_compression_algorithm diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 87c2331ee242..0e254ca711cb 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -1459,6 +1459,7 @@ def test_plan_compression(self): finally: self.connect._client._plan_compression_threshold = default_plan_compression_threshold + class SparkConnectGCTests(SparkConnectSQLTestCase): @classmethod def setUpClass(cls): From c326928fdb54227bd636574ec48b491a4f6c52eb Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Tue, 4 Nov 2025 14:28:15 +0000 Subject: [PATCH 05/19] Regen proto --- python/pyspark/sql/connect/proto/base_pb2.py | 384 ++++++++++--------- 1 file changed, 195 insertions(+), 189 deletions(-) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 0fe992332de7..fa5f418cf45a 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -45,7 +45,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"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"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"\x81\x1b\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\x8d\x01\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\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\x16newServerSideSessionId2\x8d\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\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"\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"\x81\x1b\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\x8d\x01\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\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*Q\n\x10\x43ompressionCodec\x12!\n\x1d\x43OMPRESSION_CODEC_UNSPECIFIED\x10\x00\x12\x1a\n\x16\x43OMPRESSION_CODEC_ZSTD\x10\x01\x32\x8d\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\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -70,200 +70,206 @@ _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" ]._serialized_options = b"8\001" - _globals["_PLAN"]._serialized_start = 274 - _globals["_PLAN"]._serialized_end = 390 - _globals["_USERCONTEXT"]._serialized_start = 392 - _globals["_USERCONTEXT"]._serialized_end = 514 - _globals["_ANALYZEPLANREQUEST"]._serialized_start = 517 - _globals["_ANALYZEPLANREQUEST"]._serialized_end = 3194 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_start = 1879 - _globals["_ANALYZEPLANREQUEST_SCHEMA"]._serialized_end = 1928 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_start = 1931 - _globals["_ANALYZEPLANREQUEST_EXPLAIN"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_start = 2074 - _globals["_ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE"]._serialized_end = 2246 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_start = 2248 - _globals["_ANALYZEPLANREQUEST_TREESTRING"]._serialized_end = 2338 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_start = 2340 - _globals["_ANALYZEPLANREQUEST_ISLOCAL"]._serialized_end = 2390 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_start = 2392 - _globals["_ANALYZEPLANREQUEST_ISSTREAMING"]._serialized_end = 2446 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_start = 2448 - _globals["_ANALYZEPLANREQUEST_INPUTFILES"]._serialized_end = 2501 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_start = 2503 - _globals["_ANALYZEPLANREQUEST_SPARKVERSION"]._serialized_end = 2517 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_start = 2519 - _globals["_ANALYZEPLANREQUEST_DDLPARSE"]._serialized_end = 2560 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_start = 2562 - _globals["_ANALYZEPLANREQUEST_SAMESEMANTICS"]._serialized_end = 2683 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_start = 2685 - _globals["_ANALYZEPLANREQUEST_SEMANTICHASH"]._serialized_end = 2740 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANREQUEST_PERSIST"]._serialized_end = 2894 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANREQUEST_UNPERSIST"]._serialized_end = 3006 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_start = 3008 - _globals["_ANALYZEPLANREQUEST_GETSTORAGELEVEL"]._serialized_end = 3078 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_start = 3080 - _globals["_ANALYZEPLANREQUEST_JSONTODDL"]._serialized_end = 3124 - _globals["_ANALYZEPLANRESPONSE"]._serialized_start = 3197 - _globals["_ANALYZEPLANRESPONSE"]._serialized_end = 5063 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_start = 4438 - _globals["_ANALYZEPLANRESPONSE_SCHEMA"]._serialized_end = 4495 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_start = 4497 - _globals["_ANALYZEPLANRESPONSE_EXPLAIN"]._serialized_end = 4545 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_start = 4547 - _globals["_ANALYZEPLANRESPONSE_TREESTRING"]._serialized_end = 4592 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_start = 4594 - _globals["_ANALYZEPLANRESPONSE_ISLOCAL"]._serialized_end = 4630 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_start = 4632 - _globals["_ANALYZEPLANRESPONSE_ISSTREAMING"]._serialized_end = 4680 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_start = 4682 - _globals["_ANALYZEPLANRESPONSE_INPUTFILES"]._serialized_end = 4716 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_start = 4718 - _globals["_ANALYZEPLANRESPONSE_SPARKVERSION"]._serialized_end = 4758 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_start = 4760 - _globals["_ANALYZEPLANRESPONSE_DDLPARSE"]._serialized_end = 4819 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_start = 4821 - _globals["_ANALYZEPLANRESPONSE_SAMESEMANTICS"]._serialized_end = 4860 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_start = 4862 - _globals["_ANALYZEPLANRESPONSE_SEMANTICHASH"]._serialized_end = 4900 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_start = 2743 - _globals["_ANALYZEPLANRESPONSE_PERSIST"]._serialized_end = 2752 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_start = 2896 - _globals["_ANALYZEPLANRESPONSE_UNPERSIST"]._serialized_end = 2907 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_start = 4926 - _globals["_ANALYZEPLANRESPONSE_GETSTORAGELEVEL"]._serialized_end = 5009 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_start = 5011 - _globals["_ANALYZEPLANRESPONSE_JSONTODDL"]._serialized_end = 5053 - _globals["_EXECUTEPLANREQUEST"]._serialized_start = 5066 - _globals["_EXECUTEPLANREQUEST"]._serialized_end = 5837 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_start = 5500 - _globals["_EXECUTEPLANREQUEST_REQUESTOPTION"]._serialized_end = 5761 - _globals["_EXECUTEPLANRESPONSE"]._serialized_start = 5840 - _globals["_EXECUTEPLANRESPONSE"]._serialized_end = 9297 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7940 - _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8011 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8014 - _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8262 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8265 - _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8360 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8692 + _globals["_COMPRESSIONCODEC"]._serialized_start = 18571 + _globals["_COMPRESSIONCODEC"]._serialized_end = 18652 + _globals["_PLAN"]._serialized_start = 275 + _globals["_PLAN"]._serialized_end = 758 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_start = 477 + _globals["_PLAN_COMPRESSEDOPERATION"]._serialized_end = 747 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_start = 671 + _globals["_PLAN_COMPRESSEDOPERATION_OPTYPE"]._serialized_end = 747 + _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 = 9665 + _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[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 8569 + ]._serialized_start = 8937 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_end = 8692 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8694 - _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8782 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8785 - _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8926 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8928 - _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8944 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8947 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9280 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9103 - _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9280 - _globals["_KEYVALUE"]._serialized_start = 9299 - _globals["_KEYVALUE"]._serialized_end = 9364 - _globals["_CONFIGREQUEST"]._serialized_start = 9367 - _globals["_CONFIGREQUEST"]._serialized_end = 10566 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9675 - _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10173 - _globals["_CONFIGREQUEST_SET"]._serialized_start = 10175 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 10267 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 10269 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 10294 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10296 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10359 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10361 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10392 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10394 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10442 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10444 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10471 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10473 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10507 - _globals["_CONFIGRESPONSE"]._serialized_start = 10569 - _globals["_CONFIGRESPONSE"]._serialized_end = 10744 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10747 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11749 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11222 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11275 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11277 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11388 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11390 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11483 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11486 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11679 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11752 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12024 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11943 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12024 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12027 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12353 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12356 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12708 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12551 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12666 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12668 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12708 - _globals["_INTERRUPTREQUEST"]._serialized_start = 12711 - _globals["_INTERRUPTREQUEST"]._serialized_end = 13314 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13114 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13242 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 13317 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 13461 - _globals["_REATTACHOPTIONS"]._serialized_start = 13463 - _globals["_REATTACHOPTIONS"]._serialized_end = 13516 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13519 - _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 13700 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 13703 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14109 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14112 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 14697 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14566 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14578 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14580 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14627 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 14700 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14865 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14868 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15080 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15082 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15190 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15193 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15525 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15528 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17537 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 15757 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15931 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15934 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16265 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16302 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16305 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 16855 + ]._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 = 9294 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 9296 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 9312 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 9315 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9648 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9471 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9648 + _globals["_KEYVALUE"]._serialized_start = 9667 + _globals["_KEYVALUE"]._serialized_end = 9732 + _globals["_CONFIGREQUEST"]._serialized_start = 9735 + _globals["_CONFIGREQUEST"]._serialized_end = 10934 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 10043 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10541 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 10543 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 10635 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 10637 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 10662 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10664 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10727 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10729 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10760 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10762 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10810 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10812 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10839 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10841 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10875 + _globals["_CONFIGRESPONSE"]._serialized_start = 10937 + _globals["_CONFIGRESPONSE"]._serialized_end = 11112 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 11115 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 12117 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11590 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11643 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11645 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11756 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11758 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11851 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11854 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 12047 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 12120 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12392 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12311 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12392 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12395 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12721 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12724 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 13076 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12919 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 13034 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 13036 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 13076 + _globals["_INTERRUPTREQUEST"]._serialized_start = 13079 + _globals["_INTERRUPTREQUEST"]._serialized_end = 13682 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13482 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13610 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 13685 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 13829 + _globals["_REATTACHOPTIONS"]._serialized_start = 13831 + _globals["_REATTACHOPTIONS"]._serialized_end = 13884 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13887 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 14068 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 14071 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14477 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14480 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 15065 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14934 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14946 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14948 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14995 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 15068 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 15233 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 15236 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15448 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15450 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15558 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15561 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15893 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15896 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17905 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 16125 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16299 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16302 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16633 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16670 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16673 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 17223 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 16732 + ]._serialized_start = 17100 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 16800 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 16858 - _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17108 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17110 - _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17168 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17171 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17518 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17539 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17629 - _globals["_CLONESESSIONREQUEST"]._serialized_start = 17632 - _globals["_CLONESESSIONREQUEST"]._serialized_end = 17994 - _globals["_CLONESESSIONRESPONSE"]._serialized_start = 17997 - _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18201 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 18204 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 19241 + ]._serialized_end = 17168 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 17226 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17476 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17478 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17536 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17539 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17886 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17907 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17997 + _globals["_CLONESESSIONREQUEST"]._serialized_start = 18000 + _globals["_CLONESESSIONREQUEST"]._serialized_end = 18362 + _globals["_CLONESESSIONRESPONSE"]._serialized_start = 18365 + _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18569 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 18655 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 19692 # @@protoc_insertion_point(module_scope) From 862894272c9a347c42f693d19ceeb0a779435772 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Wed, 5 Nov 2025 10:30:34 +0000 Subject: [PATCH 06/19] Add missing files --- .../ConnectPlanCompressionAlgorithm.scala | 21 +++ .../connect/utils/PlanCompressionUtils.scala | 130 ++++++++++++++++++ 2 files changed, 151 insertions(+) create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala create mode 100644 sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala new file mode 100644 index 000000000000..0f9b959ee725 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/config/ConnectPlanCompressionAlgorithm.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.connect.config + +object ConnectPlanCompressionAlgorithm extends Enumeration { + val ZSTD = Value +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala new file mode 100644 index 000000000000..c21cbd24e6e2 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.connect.utils + +import java.io.IOException + +import scala.util.control.NonFatal + +import com.github.luben.zstd.{Zstd, ZstdInputStreamNoFinalizer} +import com.google.protobuf.{ByteString, CodedInputStream} +import org.apache.commons.io.input.BoundedInputStream + +import org.apache.spark.{SparkEnv, SparkSQLException} +import org.apache.spark.connect.proto +import org.apache.spark.sql.connect.config.Connect +import org.apache.spark.sql.connect.planner.InvalidInputErrors + +object PlanCompressionUtils { + def decompressPlan(plan: proto.Plan): proto.Plan = { + plan.getOpTypeCase match { + case proto.Plan.OpTypeCase.COMPRESSED_OPERATION => + val (cis, closeStream) = decompressBytes( + plan.getCompressedOperation.getData, + plan.getCompressedOperation.getCompressionCodec + ) + try { + plan.getCompressedOperation.getOpType match { + case proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION => + proto.Plan.newBuilder().setRoot( + proto.Relation.parser().parseFrom(cis) + ).build() + case proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND => + proto.Plan.newBuilder().setCommand( + proto.Command.parser().parseFrom(cis) + ).build() + case other => + throw InvalidInputErrors.invalidOneOfField( + other, + plan.getCompressedOperation.getDescriptorForType + ) + } + } catch { + case e: SparkSQLException => + throw e + case NonFatal(e) => + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.CANNOT_PARSE", + messageParameters = Map("errorMsg" -> e.getMessage) + ) + } finally { + try { + closeStream() + } catch { + case NonFatal(_) => + } + } + case _ => plan + } + } + + private def getMaxPlanSize: Long = { + SparkEnv.get.conf.get(Connect.CONNECT_MAX_PLAN_SIZE) + } + + /** + * Decompress the given bytes using the specified codec. + * @return A tuple of decompressed CodedInputStream and a function to close the underlying stream. + */ + private def decompressBytes( + data: ByteString, + compressionCodec: proto.CompressionCodec): (CodedInputStream, () => Unit) = { + compressionCodec match { + case proto.CompressionCodec.COMPRESSION_CODEC_ZSTD => + decompressBytesWithZstd(data, getMaxPlanSize) + case other => + throw InvalidInputErrors.invalidEnum(other) + } + } + + private def decompressBytesWithZstd( + input: ByteString, maxOutputSize: Long + ): (CodedInputStream, () => Unit) = { + // Check the declared size in the header against the limit. + val declaredSize = Zstd.getFrameContentSize(input.asReadOnlyByteBuffer()) + if (declaredSize > maxOutputSize) { + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", + messageParameters = Map( + "planSize" -> declaredSize.toString, + "maxPlanSize" -> maxOutputSize.toString + ) + ) + } + + val zstdStream = new ZstdInputStreamNoFinalizer(input.newInput()) + + // Create a bounded input stream to limit the decompressed output size to avoid decompression + // bomb attacks. + val boundedStream = new BoundedInputStream(zstdStream, maxOutputSize) { + @throws[IOException] + override protected def onMaxLength(maxBytes: Long, count: Long): Unit = + throw new SparkSQLException( + errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", + messageParameters = Map( + "planSize" -> "unknown", + "maxPlanSize" -> maxOutputSize.toString + ) + ) + } + val cis = CodedInputStream.newInstance(boundedStream) + cis.setSizeLimit(Integer.MAX_VALUE) + cis.setRecursionLimit(SparkEnv.get.conf.get(Connect.CONNECT_GRPC_MARSHALLER_RECURSION_LIMIT)) + (cis, () => boundedStream.close()) + } +} From 891d479fcb260df5c32e2dc820ca04d74d8971ab Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 10:13:29 +0000 Subject: [PATCH 07/19] Add missing dependency --- dev/requirements.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/requirements.txt b/dev/requirements.txt index 76652df74481..623e551b9c95 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -65,6 +65,7 @@ grpcio>=1.67.0 grpcio-status>=1.67.0 googleapis-common-protos>=1.65.0 protobuf==5.29.5 +zstandard>=0.23.0 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 From 7b4d46b35feb2380f940d3d4e45ee71a8bda9bcd Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 10:21:44 +0000 Subject: [PATCH 08/19] Add missing dependency in spark-test-image --- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- dev/spark-test-image/python-minimum/Dockerfile | 2 +- dev/spark-test-image/python-ps-minimum/Dockerfile | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index d0409e61a51a..5d46d6f62a4e 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=21.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index ef59237afee2..c591131cccfc 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index 25fd065753bd..5ddb37887e80 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index 82016bbec860..4f7516e630e9 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 7f608caab193..d1aa475fe411 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 9fd53d233ac0..a175101f1e1d 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=21.0.0 six==1.16.0 pandas==2.3.2 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index 0ba9b620bd8b..9aff7a7da19e 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==5.29.5 googleapis-common-protos==1.65.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.75.1 grpcio-status==1.71.2 protobuf==5.29.5 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 122281ec0ea1..c7d258347bd7 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20 protobuf" # Install Python 3.9 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 680697c3f2d7..3651045efd5e 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 zstandard==0.23.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 From 2dd95e537e78a48781f26db6d34b013324f62bae Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 10:33:40 +0000 Subject: [PATCH 09/19] Add more info on what the user facing resolution is --- .../utils/src/main/resources/error/error-conditions.json | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 81d2914bbdb1..57ed891087f2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -913,12 +913,16 @@ "subClass" : { "CANNOT_PARSE" : { "message" : [ - "Cannot decompress or parse the input plan ()" + "Cannot decompress or parse the input plan ()", + "This may be caused by a corrupted compressed plan.", + "To disable plan compression, set 'spark.connect.session.planCompression.threshold' to -1." ] }, "PLAN_SIZE_LARGER_THAN_MAX" : { "message" : [ - "The plan size is larger than max ( vs. )" + "The plan size is larger than max ( vs. )", + "This typically occurs when building very complex queries with many operations, large literals, or deeply nested expressions.", + "Consider splitting the query into smaller parts using temporary views for intermediate results or reducing the number of operations." ] } }, From 9b01f337e36d45a938675bbecf8eea59e2871680 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 14:06:54 +0000 Subject: [PATCH 10/19] Reformat --- .../connect/utils/PlanCompressionUtils.scala | 48 +++++++------------ 1 file changed, 18 insertions(+), 30 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala index c21cbd24e6e2..708ef1ee6558 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/PlanCompressionUtils.scala @@ -36,23 +36,17 @@ object PlanCompressionUtils { case proto.Plan.OpTypeCase.COMPRESSED_OPERATION => val (cis, closeStream) = decompressBytes( plan.getCompressedOperation.getData, - plan.getCompressedOperation.getCompressionCodec - ) + plan.getCompressedOperation.getCompressionCodec) try { plan.getCompressedOperation.getOpType match { case proto.Plan.CompressedOperation.OpType.OP_TYPE_RELATION => - proto.Plan.newBuilder().setRoot( - proto.Relation.parser().parseFrom(cis) - ).build() + proto.Plan.newBuilder().setRoot(proto.Relation.parser().parseFrom(cis)).build() case proto.Plan.CompressedOperation.OpType.OP_TYPE_COMMAND => - proto.Plan.newBuilder().setCommand( - proto.Command.parser().parseFrom(cis) - ).build() + proto.Plan.newBuilder().setCommand(proto.Command.parser().parseFrom(cis)).build() case other => throw InvalidInputErrors.invalidOneOfField( other, - plan.getCompressedOperation.getDescriptorForType - ) + plan.getCompressedOperation.getDescriptorForType) } } catch { case e: SparkSQLException => @@ -60,14 +54,13 @@ object PlanCompressionUtils { case NonFatal(e) => throw new SparkSQLException( errorClass = "CONNECT_INVALID_PLAN.CANNOT_PARSE", - messageParameters = Map("errorMsg" -> e.getMessage) - ) + messageParameters = Map("errorMsg" -> e.getMessage)) } finally { - try { - closeStream() - } catch { - case NonFatal(_) => - } + try { + closeStream() + } catch { + case NonFatal(_) => + } } case _ => plan } @@ -79,7 +72,8 @@ object PlanCompressionUtils { /** * Decompress the given bytes using the specified codec. - * @return A tuple of decompressed CodedInputStream and a function to close the underlying stream. + * @return + * A tuple of decompressed CodedInputStream and a function to close the underlying stream. */ private def decompressBytes( data: ByteString, @@ -93,18 +87,15 @@ object PlanCompressionUtils { } private def decompressBytesWithZstd( - input: ByteString, maxOutputSize: Long - ): (CodedInputStream, () => Unit) = { + input: ByteString, + maxOutputSize: Long): (CodedInputStream, () => Unit) = { // Check the declared size in the header against the limit. val declaredSize = Zstd.getFrameContentSize(input.asReadOnlyByteBuffer()) if (declaredSize > maxOutputSize) { throw new SparkSQLException( errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", - messageParameters = Map( - "planSize" -> declaredSize.toString, - "maxPlanSize" -> maxOutputSize.toString - ) - ) + messageParameters = + Map("planSize" -> declaredSize.toString, "maxPlanSize" -> maxOutputSize.toString)) } val zstdStream = new ZstdInputStreamNoFinalizer(input.newInput()) @@ -116,11 +107,8 @@ object PlanCompressionUtils { override protected def onMaxLength(maxBytes: Long, count: Long): Unit = throw new SparkSQLException( errorClass = "CONNECT_INVALID_PLAN.PLAN_SIZE_LARGER_THAN_MAX", - messageParameters = Map( - "planSize" -> "unknown", - "maxPlanSize" -> maxOutputSize.toString - ) - ) + messageParameters = + Map("planSize" -> "unknown", "maxPlanSize" -> maxOutputSize.toString)) } val cis = CodedInputStream.newInstance(boundedStream) cis.setSizeLimit(Integer.MAX_VALUE) From f18d3424a4f7b4ab95c531c2ba0fbcca2f8cc507 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Sat, 8 Nov 2025 22:00:48 +0000 Subject: [PATCH 11/19] Reformat --- python/pyspark/sql/connect/client/core.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 07136d3d1513..2748558ba733 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -1984,7 +1984,8 @@ def _handle_rpc_error(self, rpc_error: grpc.RpcError) -> NoReturn: if info.metadata.get("errorClass") == "CONNECT_INVALID_PLAN.CANNOT_PARSE": # Disable plan compression if the server fails to interpret the plan. logger.info( - "Disabling plan compression for the session due to CONNECT_INVALID_PLAN.CANNOT_PARSE error." + "Disabling plan compression for the session due to " + "CONNECT_INVALID_PLAN.CANNOT_PARSE error." ) self._plan_compression_threshold, self._plan_compression_algorithm = ( -1, @@ -2194,8 +2195,10 @@ def _try_compress_and_set_plan( duration = time.time() - start_time compressed_size = len(compressed_operation.data) logger.debug( - f"Plan compression: original_size={original_size}, compressed_size={compressed_size}, " - f"saving_ratio={1 - compressed_size / original_size:.2f}, duration_s={duration:.1f}" + f"Plan compression: original_size={original_size}, " + f"compressed_size={compressed_size}, " + f"saving_ratio={1 - compressed_size / original_size:.2f}, " + f"duration_s={duration:.1f}" ) if compressed_size < original_size: plan.compressed_operation.CopyFrom(compressed_operation) From 7c4cae19f7abc5130bc098bd4f5a138251fbd8a3 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 09:07:24 +0000 Subject: [PATCH 12/19] Fix lint CI error --- dev/spark-test-image/lint/Dockerfile | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 4dfceae63a17..14afde4de4ac 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -84,6 +84,7 @@ RUN python3.11 -m pip install \ 'grpc-stubs==1.24.11' \ 'grpcio-status==1.76.0' \ 'grpcio==1.76.0' \ + 'zstandard==0.23.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ From d5e820c1f31d5099e415267db3da19682a44de38 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 14:38:51 +0000 Subject: [PATCH 13/19] Fix lint --- python/pyspark/sql/connect/client/core.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 2748558ba733..06972030b056 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -114,7 +114,7 @@ from pyspark.sql.datasource import DataSource -def _import_zstandard_if_available(): +def _import_zstandard_if_available() -> Optional[Any]: """ Import zstandard if available, otherwise return None. This is used to handle the case when zstandard is not installed. @@ -721,8 +721,8 @@ def __init__( self._progress_handlers: List[ProgressHandler] = [] self._zstd_module = _import_zstandard_if_available() - self._plan_compression_threshold = None # Will be fetched lazily - self._plan_compression_algorithm = None # Will be fetched lazily + self._plan_compression_threshold: Optional[int] = None # Will be fetched lazily + self._plan_compression_algorithm: Optional[str] = None # Will be fetched lazily # cleanup ml cache if possible atexit.register(self._cleanup_ml_cache) @@ -2161,7 +2161,7 @@ def _try_compress_and_set_plan( self, plan: pb2.Plan, message: google.protobuf.message.Message, - op_type: pb2.Plan.CompressedOperation.OpType, + op_type: pb2.Plan.CompressedOperation.OpType.ValueType, ) -> None: """ Tries to compress a protobuf message and sets it on the plan. From 1dbf99d1bf4585a65e061e7d2022601c9f72cae9 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 14:42:04 +0000 Subject: [PATCH 14/19] Fix lint --- python/pyspark/sql/connect/client/core.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 06972030b056..8eaaeda26a7d 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2207,9 +2207,9 @@ def _try_compress_and_set_plan( logger.debug("Plan compression not effective. Using original plan.") if op_type == pb2.Plan.CompressedOperation.OpType.OP_TYPE_RELATION: - plan.root.CopyFrom(message) + plan.root.CopyFrom(message) # type: ignore[arg-type] else: - plan.command.CopyFrom(message) + plan.command.CopyFrom(message) # type: ignore[arg-type] def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: if self._plan_compression_threshold is None or self._plan_compression_algorithm is None: @@ -2232,7 +2232,7 @@ def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: logger.debug( "Plan compression is disabled because the server does not support it.", e ) - return self._plan_compression_threshold, self._plan_compression_algorithm + return self._plan_compression_threshold, self._plan_compression_algorithm # type: ignore[return-value] def clone(self, new_session_id: Optional[str] = None) -> "SparkConnectClient": """ From cea703f0fdda9f78d9ffff38b96ad63cee4023bc Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 15:49:17 +0000 Subject: [PATCH 15/19] Fix lint CI error --- python/pyspark/sql/connect/client/core.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 8eaaeda26a7d..bd50df576f44 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2232,7 +2232,10 @@ def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: logger.debug( "Plan compression is disabled because the server does not support it.", e ) - return self._plan_compression_threshold, self._plan_compression_algorithm # type: ignore[return-value] + return ( + self._plan_compression_threshold, + self._plan_compression_algorithm, + ) # type: ignore[return-value] def clone(self, new_session_id: Optional[str] = None) -> "SparkConnectClient": """ From 9225af55562e4014da72b1ff6e6bfb7cdcbf59f7 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 19:32:49 +0000 Subject: [PATCH 16/19] Fix lint --- python/pyspark/sql/connect/client/core.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index bd50df576f44..bf9021186620 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2215,13 +2215,13 @@ def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: if self._plan_compression_threshold is None or self._plan_compression_algorithm is None: try: ( - self._plan_compression_threshold, + plan_compression_threshold_str, self._plan_compression_algorithm, ) = self.get_configs( "spark.connect.session.planCompression.threshold", "spark.connect.session.planCompression.defaultAlgorithm", ) - self._plan_compression_threshold = int(self._plan_compression_threshold) + self._plan_compression_threshold = int(plan_compression_threshold_str) logger.debug( f"Plan compression threshold: {self._plan_compression_threshold}, " f"algorithm: {self._plan_compression_algorithm}" From 9a77ef4393978b8bc44b077b729284f4ba0db000 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 20:02:34 +0000 Subject: [PATCH 17/19] Fix lint --- python/pyspark/sql/connect/client/core.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index bf9021186620..6637fa026b13 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2221,7 +2221,9 @@ def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: "spark.connect.session.planCompression.threshold", "spark.connect.session.planCompression.defaultAlgorithm", ) - self._plan_compression_threshold = int(plan_compression_threshold_str) + self._plan_compression_threshold = int( + plan_compression_threshold_str + ) # type: ignore[arg-type] logger.debug( f"Plan compression threshold: {self._plan_compression_threshold}, " f"algorithm: {self._plan_compression_algorithm}" From 2c93b7a13faf8d0679d881b99102da2ea8d35d21 Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Mon, 10 Nov 2025 20:17:29 +0000 Subject: [PATCH 18/19] Fix lint --- python/pyspark/sql/connect/client/core.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 6637fa026b13..6bff531c23d4 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2221,9 +2221,9 @@ def _get_plan_compression_threshold_and_algorithm(self) -> Tuple[int, str]: "spark.connect.session.planCompression.threshold", "spark.connect.session.planCompression.defaultAlgorithm", ) - self._plan_compression_threshold = int( - plan_compression_threshold_str - ) # type: ignore[arg-type] + self._plan_compression_threshold = ( + int(plan_compression_threshold_str) if plan_compression_threshold_str else -1 + ) logger.debug( f"Plan compression threshold: {self._plan_compression_threshold}, " f"algorithm: {self._plan_compression_algorithm}" From c39bdced20548d7e6b386d6da999e293c3ff34af Mon Sep 17 00:00:00 2001 From: Xi Lyu Date: Tue, 11 Nov 2025 09:20:49 +0000 Subject: [PATCH 19/19] Upgrade the version of the new dependency zstandard to the latest stable version 0.25.0 --- dev/requirements.txt | 2 +- dev/spark-test-image/lint/Dockerfile | 2 +- dev/spark-test-image/numpy-213/Dockerfile | 2 +- dev/spark-test-image/python-310/Dockerfile | 2 +- dev/spark-test-image/python-311/Dockerfile | 2 +- dev/spark-test-image/python-312/Dockerfile | 2 +- dev/spark-test-image/python-313-nogil/Dockerfile | 2 +- dev/spark-test-image/python-313/Dockerfile | 2 +- dev/spark-test-image/python-314/Dockerfile | 2 +- dev/spark-test-image/python-minimum/Dockerfile | 2 +- dev/spark-test-image/python-ps-minimum/Dockerfile | 2 +- python/packaging/classic/setup.py | 2 +- python/packaging/client/setup.py | 2 +- python/packaging/connect/setup.py | 2 +- python/pyspark/sql/connect/utils.py | 2 +- 15 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dev/requirements.txt b/dev/requirements.txt index 78e0c3b23aaa..cde0957715bf 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -65,7 +65,7 @@ grpcio>=1.76.0 grpcio-status>=1.76.0 googleapis-common-protos>=1.71.0 protobuf==6.33.0 -zstandard>=0.23.0 +zstandard>=0.25.0 # Spark Connect python proto generation plugin (optional) mypy-protobuf==3.3.0 diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index 14afde4de4ac..6ab571bf35d6 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -84,7 +84,7 @@ RUN python3.11 -m pip install \ 'grpc-stubs==1.24.11' \ 'grpcio-status==1.76.0' \ 'grpcio==1.76.0' \ - 'zstandard==0.23.0' \ + 'zstandard==0.25.0' \ 'ipython' \ 'ipython_genutils' \ 'jinja2' \ diff --git a/dev/spark-test-image/numpy-213/Dockerfile b/dev/spark-test-image/numpy-213/Dockerfile index 2a48fb7294a0..713e9e7d7ef4 100644 --- a/dev/spark-test-image/numpy-213/Dockerfile +++ b/dev/spark-test-image/numpy-213/Dockerfile @@ -71,7 +71,7 @@ RUN apt-get update && apt-get install -y \ # Pin numpy==2.1.3 ARG BASIC_PIP_PKGS="numpy==2.1.3 pyarrow>=22.0.0 six==1.16.0 pandas==2.2.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-310/Dockerfile b/dev/spark-test-image/python-310/Dockerfile index 5e4370cee3fe..9b5b18d061c2 100644 --- a/dev/spark-test-image/python-310/Dockerfile +++ b/dev/spark-test-image/python-310/Dockerfile @@ -66,7 +66,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-311/Dockerfile b/dev/spark-test-image/python-311/Dockerfile index f29665e32faa..f8a9df5842ce 100644 --- a/dev/spark-test-image/python-311/Dockerfile +++ b/dev/spark-test-image/python-311/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.11 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.11 diff --git a/dev/spark-test-image/python-312/Dockerfile b/dev/spark-test-image/python-312/Dockerfile index ae9e781303f0..ca62bc5ebc61 100644 --- a/dev/spark-test-image/python-312/Dockerfile +++ b/dev/spark-test-image/python-312/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.12 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.12 diff --git a/dev/spark-test-image/python-313-nogil/Dockerfile b/dev/spark-test-image/python-313-nogil/Dockerfile index 29cb9e621920..b6e2dd7c80a9 100644 --- a/dev/spark-test-image/python-313-nogil/Dockerfile +++ b/dev/spark-test-image/python-313-nogil/Dockerfile @@ -69,7 +69,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.13 packages diff --git a/dev/spark-test-image/python-313/Dockerfile b/dev/spark-test-image/python-313/Dockerfile index 0edc5c294744..bd64ecb31087 100644 --- a/dev/spark-test-image/python-313/Dockerfile +++ b/dev/spark-test-image/python-313/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.13 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.13 diff --git a/dev/spark-test-image/python-314/Dockerfile b/dev/spark-test-image/python-314/Dockerfile index a03c435c88f7..f3da21e005b3 100644 --- a/dev/spark-test-image/python-314/Dockerfile +++ b/dev/spark-test-image/python-314/Dockerfile @@ -70,7 +70,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy pyarrow>=22.0.0 six==1.16.0 pandas==2.3.3 scipy plotly<6.0.0 mlflow>=2.8.1 coverage matplotlib openpyxl memory-profiler>=0.61.0 scikit-learn>=1.3.2" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20.3" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 protobuf==6.33.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20.3" # Install Python 3.14 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.14 diff --git a/dev/spark-test-image/python-minimum/Dockerfile b/dev/spark-test-image/python-minimum/Dockerfile index 33ed88b621ac..575b4afdd02c 100644 --- a/dev/spark-test-image/python-minimum/Dockerfile +++ b/dev/spark-test-image/python-minimum/Dockerfile @@ -64,7 +64,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="numpy==1.22.4 pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 scipy scikit-learn coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile index 333e42b18f92..5142d46cc3eb 100644 --- a/dev/spark-test-image/python-ps-minimum/Dockerfile +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -65,7 +65,7 @@ RUN apt-get update && apt-get install -y \ ARG BASIC_PIP_PKGS="pyarrow==15.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" # Python deps for Spark Connect -ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.23.0 graphviz==0.20 protobuf" +ARG CONNECT_PIP_PKGS="grpcio==1.76.0 grpcio-status==1.76.0 googleapis-common-protos==1.71.0 zstandard==0.25.0 graphviz==0.20 protobuf" # Install Python 3.10 packages RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.10 diff --git a/python/packaging/classic/setup.py b/python/packaging/classic/setup.py index d3a2025b8c3b..54ec4abe3be9 100755 --- a/python/packaging/classic/setup.py +++ b/python/packaging/classic/setup.py @@ -156,7 +156,7 @@ def _supports_symlinks(): _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" -_minimum_zstandard_version = "0.23.0" +_minimum_zstandard_version = "0.25.0" class InstallCommand(install): diff --git a/python/packaging/client/setup.py b/python/packaging/client/setup.py index 1030cfce4e56..ee404210f293 100755 --- a/python/packaging/client/setup.py +++ b/python/packaging/client/setup.py @@ -139,7 +139,7 @@ _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" - _minimum_zstandard_version = "0.23.0" + _minimum_zstandard_version = "0.25.0" with open("README.md") as f: long_description = f.read() diff --git a/python/packaging/connect/setup.py b/python/packaging/connect/setup.py index 12bc1ee68ca1..9a1a4ea81255 100755 --- a/python/packaging/connect/setup.py +++ b/python/packaging/connect/setup.py @@ -92,7 +92,7 @@ _minimum_grpc_version = "1.76.0" _minimum_googleapis_common_protos_version = "1.71.0" _minimum_pyyaml_version = "3.11" - _minimum_zstandard_version = "0.23.0" + _minimum_zstandard_version = "0.25.0" with open("README.md") as f: long_description = f.read() diff --git a/python/pyspark/sql/connect/utils.py b/python/pyspark/sql/connect/utils.py index bc29a1fd67bc..0e0e04244653 100644 --- a/python/pyspark/sql/connect/utils.py +++ b/python/pyspark/sql/connect/utils.py @@ -99,7 +99,7 @@ def require_minimum_googleapis_common_protos_version() -> None: def require_minimum_zstandard_version() -> None: """Raise ImportError if zstandard is not installed""" - minimum_zstandard_version = "0.23.0" + minimum_zstandard_version = "0.25.0" try: import zstandard # noqa