diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 414781d67cd4..0cb257ae8fb4 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -1194,6 +1194,115 @@ def execute_command_as_iterator( }, ) + def batch_execute( + self, + plan_sequences: List[List[Tuple[pb2.Plan, Optional[str]]]], + sequence_operation_ids: Optional[List[Optional[str]]] = None, + ) -> pb2.BatchExecutePlanResponse: + """ + Execute multiple sequences of plans in batch. + + Each sequence executes sequentially, all sequences execute in parallel. + Single-plan batches are treated as sequences containing one plan. + + Parameters + ---------- + plan_sequences : list of list of (Plan, Optional[str]) + List of sequences. Each sequence is a list of (plan, operation_id) tuples. + sequence_operation_ids : list of Optional[str], optional + Optional operation IDs for each sequence + + Returns + ------- + BatchExecutePlanResponse + Response containing sequence operation IDs and query operation IDs + """ + import uuid + + req = pb2.BatchExecutePlanRequest() + req.session_id = self._session_id + if self._user_id: + req.user_context.user_id = self._user_id + req.client_type = self._builder.userAgent + + if self._server_side_session_id: + req.client_observed_server_side_session_id = self._server_side_session_id + + seq_op_ids = sequence_operation_ids or [] + + for idx, sequence in enumerate(plan_sequences): + seq = req.plan_sequences.add() + + # Set sequence operation ID if provided + if idx < len(seq_op_ids) and seq_op_ids[idx]: + try: + uuid.UUID(seq_op_ids[idx]) + seq.sequence_operation_id = seq_op_ids[idx] + except ValueError: + raise PySparkValueError( + error_class="INVALID_HANDLE.FORMAT", + message_parameters={"handle": seq_op_ids[idx]}, + ) + + # Add plans to sequence + for plan, op_id in sequence: + plan_exec = seq.plan_executions.add() + plan_exec.plan.CopyFrom(plan) + if op_id: + try: + uuid.UUID(op_id) + plan_exec.operation_id = op_id + except ValueError: + raise PySparkValueError( + error_class="INVALID_HANDLE.FORMAT", + message_parameters={"handle": op_id}, + ) + + metadata = self._builder.metadata() + return self._stub.BatchExecutePlan(req, metadata=metadata) + + def reattach_execute(self, operation_id: str) -> Iterator[pb2.ExecutePlanResponse]: + """ + Reattach to an existing operation by operation ID and consume all responses. + + Parameters + ---------- + operation_id : str + The operation ID to reattach to (must be a valid UUID) + + Returns + ------- + Iterator[pb2.ExecutePlanResponse] + An iterator of ExecutePlanResponse messages + + Raises + ------ + ValueError + If the operation_id is not a valid UUID format + """ + import uuid + + try: + uuid.UUID(operation_id) + except ValueError: + raise PySparkValueError( + error_class="INVALID_HANDLE.FORMAT", + message_parameters={"handle": operation_id}, + ) + + req = pb2.ReattachExecuteRequest() + req.session_id = self._session_id + if self._user_id: + req.user_context.user_id = self._user_id + req.operation_id = operation_id + req.client_type = self._builder.userAgent + + if self._server_side_session_id: + req.client_observed_server_side_session_id = self._server_side_session_id + + metadata = self._builder.metadata() + return self._stub.ReattachExecute(req, metadata=metadata) + def same_semantics(self, plan: pb2.Plan, other: pb2.Plan) -> bool: """ return if two plans have the same semantics. diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 0fe992332de7..cde742b97626 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"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"\xcb\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\x12\x31\n\x12query_operation_id\x18\x18 \x01(\tH\x01R\x10queryOperationId\x88\x01\x01\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_typeB\x15\n\x13_query_operation_id"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"\xd7\x05\n\x17\x42\x61tchExecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x02 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x03 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12Z\n\x0eplan_sequences\x18\x05 \x03(\x0b\x32\x33.spark.connect.BatchExecutePlanRequest.PlanSequenceR\rplanSequences\x1a\xc0\x01\n\x0cPlanSequence\x12\x37\n\x15sequence_operation_id\x18\x01 \x01(\tH\x00R\x13sequenceOperationId\x88\x01\x01\x12]\n\x0fplan_executions\x18\x02 \x03(\x0b\x32\x34.spark.connect.BatchExecutePlanRequest.PlanExecutionR\x0eplanExecutionsB\x18\n\x16_sequence_operation_id\x1a\x85\x01\n\rPlanExecution\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x12\x12\n\x04tags\x18\x03 \x03(\tR\x04tagsB\x0f\n\r_operation_idB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xb0\x04\n\x18\x42\x61tchExecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId\x12\x61\n\x10sequence_results\x18\x03 \x03(\x0b\x32\x36.spark.connect.BatchExecutePlanResponse.SequenceResultR\x0fsequenceResults\x1a\x84\x02\n\x0eSequenceResult\x12\x32\n\x15sequence_operation_id\x18\x01 \x01(\tR\x13sequenceOperationId\x12\x18\n\x07success\x18\x02 \x01(\x08R\x07success\x12(\n\rerror_message\x18\x03 \x01(\tH\x00R\x0c\x65rrorMessage\x88\x01\x01\x12h\n\x13query_operation_ids\x18\x04 \x03(\x0b\x32\x38.spark.connect.BatchExecutePlanResponse.QueryOperationIdR\x11queryOperationIdsB\x10\n\x0e_error_message\x1aV\n\x10QueryOperationId\x12!\n\x0coperation_id\x18\x01 \x01(\tR\x0boperationId\x12\x1f\n\x0bquery_index\x18\x02 \x01(\x05R\nqueryIndex2\xf4\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\x12\x65\n\x10\x42\x61tchExecutePlan\x12&.spark.connect.BatchExecutePlanRequest\x1a\'.spark.connect.BatchExecutePlanResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -141,129 +141,141 @@ _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["_EXECUTEPLANRESPONSE"]._serialized_end = 9371 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_start = 7991 + _globals["_EXECUTEPLANRESPONSE_SQLCOMMANDRESULT"]._serialized_end = 8062 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_start = 8065 + _globals["_EXECUTEPLANRESPONSE_ARROWBATCH"]._serialized_end = 8313 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_start = 8316 + _globals["_EXECUTEPLANRESPONSE_METRICS"]._serialized_end = 8833 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_start = 8411 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT"]._serialized_end = 8743 _globals[ "_EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY" - ]._serialized_start = 8569 + ]._serialized_start = 8620 _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 = 8743 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_start = 8745 + _globals["_EXECUTEPLANRESPONSE_METRICS_METRICVALUE"]._serialized_end = 8833 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_start = 8836 + _globals["_EXECUTEPLANRESPONSE_OBSERVEDMETRICS"]._serialized_end = 8977 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_start = 8979 + _globals["_EXECUTEPLANRESPONSE_RESULTCOMPLETE"]._serialized_end = 8995 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_start = 8998 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS"]._serialized_end = 9331 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_start = 9154 + _globals["_EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO"]._serialized_end = 9331 + _globals["_KEYVALUE"]._serialized_start = 9373 + _globals["_KEYVALUE"]._serialized_end = 9438 + _globals["_CONFIGREQUEST"]._serialized_start = 9441 + _globals["_CONFIGREQUEST"]._serialized_end = 10640 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 9749 + _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 10247 + _globals["_CONFIGREQUEST_SET"]._serialized_start = 10249 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 10341 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 10343 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 10368 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 10370 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 10433 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 10435 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 10466 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 10468 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 10516 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 10518 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 10545 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 10547 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 10581 + _globals["_CONFIGRESPONSE"]._serialized_start = 10643 + _globals["_CONFIGRESPONSE"]._serialized_end = 10818 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10821 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11823 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 11296 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 11349 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 11351 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 11462 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 11464 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 11557 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 11560 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 11753 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11826 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 12098 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 12017 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 12098 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 12101 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 12427 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 12430 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12782 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 12625 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 12740 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 12742 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12782 + _globals["_INTERRUPTREQUEST"]._serialized_start = 12785 + _globals["_INTERRUPTREQUEST"]._serialized_end = 13388 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 13188 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 13316 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 13391 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 13535 + _globals["_REATTACHOPTIONS"]._serialized_start = 13537 + _globals["_REATTACHOPTIONS"]._serialized_end = 13590 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_start = 13593 + _globals["_RESULTCHUNKINGOPTIONS"]._serialized_end = 13774 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 13777 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 14183 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 14186 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 14771 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 14640 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 14652 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 14654 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 14701 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 14774 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 14939 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 14942 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 15154 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 15156 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 15264 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 15267 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 15599 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 15602 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 17611 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 15831 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 16005 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 16008 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 16376 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 16339 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 16376 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 16379 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 16929 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 16732 + ]._serialized_start = 16806 _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 = 16874 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_start = 16932 + _globals["_FETCHERRORDETAILSRESPONSE_BREAKINGCHANGEINFO"]._serialized_end = 17182 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_start = 17184 + _globals["_FETCHERRORDETAILSRESPONSE_MITIGATIONCONFIG"]._serialized_end = 17242 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 17245 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 17592 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 17613 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 17703 + _globals["_CLONESESSIONREQUEST"]._serialized_start = 17706 + _globals["_CLONESESSIONREQUEST"]._serialized_end = 18068 + _globals["_CLONESESSIONRESPONSE"]._serialized_start = 18071 + _globals["_CLONESESSIONRESPONSE"]._serialized_end = 18275 + _globals["_BATCHEXECUTEPLANREQUEST"]._serialized_start = 18278 + _globals["_BATCHEXECUTEPLANREQUEST"]._serialized_end = 19005 + _globals["_BATCHEXECUTEPLANREQUEST_PLANSEQUENCE"]._serialized_start = 18618 + _globals["_BATCHEXECUTEPLANREQUEST_PLANSEQUENCE"]._serialized_end = 18810 + _globals["_BATCHEXECUTEPLANREQUEST_PLANEXECUTION"]._serialized_start = 18813 + _globals["_BATCHEXECUTEPLANREQUEST_PLANEXECUTION"]._serialized_end = 18946 + _globals["_BATCHEXECUTEPLANRESPONSE"]._serialized_start = 19008 + _globals["_BATCHEXECUTEPLANRESPONSE"]._serialized_end = 19568 + _globals["_BATCHEXECUTEPLANRESPONSE_SEQUENCERESULT"]._serialized_start = 19220 + _globals["_BATCHEXECUTEPLANRESPONSE_SEQUENCERESULT"]._serialized_end = 19480 + _globals["_BATCHEXECUTEPLANRESPONSE_QUERYOPERATIONID"]._serialized_start = 19482 + _globals["_BATCHEXECUTEPLANRESPONSE_QUERYOPERATIONID"]._serialized_end = 19568 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 19571 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 20711 # @@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 dc3099ecdffc..58a20c458fb9 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -1284,7 +1284,7 @@ global___ExecutePlanRequest = ExecutePlanRequest class ExecutePlanResponse(google.protobuf.message.Message): """The response of a query, can be one or more for each request. Responses belonging to the same input query, carry the same `session_id`. - Next ID: 17 + Next ID: 25 """ DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1623,6 +1623,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int OPERATION_ID_FIELD_NUMBER: builtins.int RESPONSE_ID_FIELD_NUMBER: builtins.int + QUERY_OPERATION_ID_FIELD_NUMBER: builtins.int ARROW_BATCH_FIELD_NUMBER: builtins.int SQL_COMMAND_RESULT_FIELD_NUMBER: builtins.int WRITE_STREAM_OPERATION_START_RESULT_FIELD_NUMBER: builtins.int @@ -1657,6 +1658,10 @@ class ExecutePlanResponse(google.protobuf.message.Message): """Identified the response in the stream. The id is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` """ + query_operation_id: builtins.str + """(Optional) For sequence executions: identifies which query in the sequence this response belongs to. + This is set when the execution is part of a BatchExecutePlan sequence. + """ @property def arrow_batch(self) -> global___ExecutePlanResponse.ArrowBatch: ... @property @@ -1744,6 +1749,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): server_side_session_id: builtins.str = ..., operation_id: builtins.str = ..., response_id: builtins.str = ..., + query_operation_id: builtins.str | None = ..., arrow_batch: global___ExecutePlanResponse.ArrowBatch | None = ..., sql_command_result: global___ExecutePlanResponse.SqlCommandResult | None = ..., write_stream_operation_start_result: pyspark.sql.connect.proto.commands_pb2.WriteStreamOperationStartResult @@ -1777,6 +1783,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): def HasField( self, field_name: typing_extensions.Literal[ + "_query_operation_id", + b"_query_operation_id", "arrow_batch", b"arrow_batch", "checkpoint_command_result", @@ -1799,6 +1807,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"pipeline_event_result", "pipeline_query_function_execution_signal", b"pipeline_query_function_execution_signal", + "query_operation_id", + b"query_operation_id", "response_type", b"response_type", "result_complete", @@ -1820,6 +1830,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ + "_query_operation_id", + b"_query_operation_id", "arrow_batch", b"arrow_batch", "checkpoint_command_result", @@ -1846,6 +1858,8 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"pipeline_event_result", "pipeline_query_function_execution_signal", b"pipeline_query_function_execution_signal", + "query_operation_id", + b"query_operation_id", "response_id", b"response_id", "response_type", @@ -1870,6 +1884,11 @@ class ExecutePlanResponse(google.protobuf.message.Message): b"write_stream_operation_start_result", ], ) -> None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_query_operation_id", b"_query_operation_id"] + ) -> typing_extensions.Literal["query_operation_id"] | None: ... + @typing.overload def WhichOneof( self, oneof_group: typing_extensions.Literal["response_type", b"response_type"] ) -> ( @@ -4230,3 +4249,319 @@ class CloneSessionResponse(google.protobuf.message.Message): ) -> None: ... global___CloneSessionResponse = CloneSessionResponse + +class BatchExecutePlanRequest(google.protobuf.message.Message): + """Request to execute multiple sequences of plans in batch. + Each sequence executes sequentially in its own thread, all sequences execute in parallel. + Single-plan batches are treated as sequences with one plan. + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class PlanSequence(google.protobuf.message.Message): + """A sequence of plans that execute sequentially""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SEQUENCE_OPERATION_ID_FIELD_NUMBER: builtins.int + PLAN_EXECUTIONS_FIELD_NUMBER: builtins.int + sequence_operation_id: builtins.str + """(Optional) Operation ID for the entire sequence. + If not provided, a UUID will be generated by the server. + The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` + """ + @property + def plan_executions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___BatchExecutePlanRequest.PlanExecution + ]: + """(Required) Plans to execute sequentially within this sequence""" + def __init__( + self, + *, + sequence_operation_id: builtins.str | None = ..., + plan_executions: collections.abc.Iterable[ + global___BatchExecutePlanRequest.PlanExecution + ] + | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_sequence_operation_id", + b"_sequence_operation_id", + "sequence_operation_id", + b"sequence_operation_id", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_sequence_operation_id", + b"_sequence_operation_id", + "plan_executions", + b"plan_executions", + "sequence_operation_id", + b"sequence_operation_id", + ], + ) -> None: ... + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "_sequence_operation_id", b"_sequence_operation_id" + ], + ) -> typing_extensions.Literal["sequence_operation_id"] | None: ... + + class PlanExecution(google.protobuf.message.Message): + """A single plan execution within a sequence""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + PLAN_FIELD_NUMBER: builtins.int + OPERATION_ID_FIELD_NUMBER: builtins.int + TAGS_FIELD_NUMBER: builtins.int + @property + def plan(self) -> global___Plan: + """(Required) The logical plan to execute.""" + operation_id: builtins.str + """(Optional) Operation ID for this individual query. + If not provided, a UUID will be generated by the server. + The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` + """ + @property + def tags( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Optional) Tags to be assigned to the operations.""" + def __init__( + self, + *, + plan: global___Plan | None = ..., + operation_id: builtins.str | None = ..., + tags: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_operation_id", b"_operation_id", "operation_id", b"operation_id", "plan", b"plan" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_operation_id", + b"_operation_id", + "operation_id", + b"operation_id", + "plan", + b"plan", + "tags", + b"tags", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_operation_id", b"_operation_id"] + ) -> typing_extensions.Literal["operation_id"] | None: ... + + SESSION_ID_FIELD_NUMBER: builtins.int + CLIENT_OBSERVED_SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int + USER_CONTEXT_FIELD_NUMBER: builtins.int + CLIENT_TYPE_FIELD_NUMBER: builtins.int + PLAN_SEQUENCES_FIELD_NUMBER: builtins.int + session_id: builtins.str + """(Required) The session_id for which this batch execute is targeting.""" + client_observed_server_side_session_id: builtins.str + """(Optional) Client-side observed server-side session ID.""" + @property + def user_context(self) -> global___UserContext: + """(Required) User context""" + client_type: builtins.str + """(Optional) An optional client-type identifier (e.g., "spark-sql", "pyspark").""" + @property + def plan_sequences( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___BatchExecutePlanRequest.PlanSequence + ]: + """(Required) List of plan sequences to execute. Each sequence runs in parallel.""" + def __init__( + self, + *, + session_id: builtins.str = ..., + client_observed_server_side_session_id: builtins.str | None = ..., + user_context: global___UserContext | None = ..., + client_type: builtins.str | None = ..., + plan_sequences: collections.abc.Iterable[global___BatchExecutePlanRequest.PlanSequence] + | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_client_observed_server_side_session_id", + b"_client_observed_server_side_session_id", + "_client_type", + b"_client_type", + "client_observed_server_side_session_id", + b"client_observed_server_side_session_id", + "client_type", + b"client_type", + "user_context", + b"user_context", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_client_observed_server_side_session_id", + b"_client_observed_server_side_session_id", + "_client_type", + b"_client_type", + "client_observed_server_side_session_id", + b"client_observed_server_side_session_id", + "client_type", + b"client_type", + "plan_sequences", + b"plan_sequences", + "session_id", + b"session_id", + "user_context", + b"user_context", + ], + ) -> None: ... + @typing.overload + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "_client_observed_server_side_session_id", b"_client_observed_server_side_session_id" + ], + ) -> typing_extensions.Literal["client_observed_server_side_session_id"] | None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_client_type", b"_client_type"] + ) -> typing_extensions.Literal["client_type"] | None: ... + +global___BatchExecutePlanRequest = BatchExecutePlanRequest + +class BatchExecutePlanResponse(google.protobuf.message.Message): + """Response for batch execute plan request.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class SequenceResult(google.protobuf.message.Message): + """Result for a single sequence execution""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SEQUENCE_OPERATION_ID_FIELD_NUMBER: builtins.int + SUCCESS_FIELD_NUMBER: builtins.int + ERROR_MESSAGE_FIELD_NUMBER: builtins.int + QUERY_OPERATION_IDS_FIELD_NUMBER: builtins.int + sequence_operation_id: builtins.str + """(Required) Operation ID for the entire sequence (reattach to this)""" + success: builtins.bool + """(Required) Whether the sequence was successfully submitted""" + error_message: builtins.str + """(Optional) Error message if submission failed""" + @property + def query_operation_ids( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___BatchExecutePlanResponse.QueryOperationId + ]: + """(Required) Operation IDs for individual queries within the sequence""" + def __init__( + self, + *, + sequence_operation_id: builtins.str = ..., + success: builtins.bool = ..., + error_message: builtins.str | None = ..., + query_operation_ids: collections.abc.Iterable[ + global___BatchExecutePlanResponse.QueryOperationId + ] + | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_error_message", b"_error_message", "error_message", b"error_message" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_error_message", + b"_error_message", + "error_message", + b"error_message", + "query_operation_ids", + b"query_operation_ids", + "sequence_operation_id", + b"sequence_operation_id", + "success", + b"success", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_error_message", b"_error_message"] + ) -> typing_extensions.Literal["error_message"] | None: ... + + class QueryOperationId(google.protobuf.message.Message): + """Query operation ID with its index in the sequence""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + OPERATION_ID_FIELD_NUMBER: builtins.int + QUERY_INDEX_FIELD_NUMBER: builtins.int + operation_id: builtins.str + """(Required) Operation ID for this individual query""" + query_index: builtins.int + """(Required) Index of this query in the sequence (0-based)""" + def __init__( + self, + *, + operation_id: builtins.str = ..., + query_index: builtins.int = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "operation_id", b"operation_id", "query_index", b"query_index" + ], + ) -> None: ... + + SESSION_ID_FIELD_NUMBER: builtins.int + SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int + SEQUENCE_RESULTS_FIELD_NUMBER: builtins.int + session_id: builtins.str + """Session id in which the batch was executed.""" + server_side_session_id: builtins.str + """Server-side generated idempotency key for the session.""" + @property + def sequence_results( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___BatchExecutePlanResponse.SequenceResult + ]: + """Results for each sequence in the same order as the request""" + def __init__( + self, + *, + session_id: builtins.str = ..., + server_side_session_id: builtins.str = ..., + sequence_results: collections.abc.Iterable[global___BatchExecutePlanResponse.SequenceResult] + | None = ..., + ) -> None: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "sequence_results", + b"sequence_results", + "server_side_session_id", + b"server_side_session_id", + "session_id", + b"session_id", + ], + ) -> None: ... + +global___BatchExecutePlanResponse = BatchExecutePlanResponse diff --git a/python/pyspark/sql/connect/proto/base_pb2_grpc.py b/python/pyspark/sql/connect/proto/base_pb2_grpc.py index 16dd20b563f3..90f5d1b395a5 100644 --- a/python/pyspark/sql/connect/proto/base_pb2_grpc.py +++ b/python/pyspark/sql/connect/proto/base_pb2_grpc.py @@ -96,6 +96,12 @@ def __init__(self, channel): response_deserializer=spark_dot_connect_dot_base__pb2.CloneSessionResponse.FromString, _registered_method=True, ) + self.BatchExecutePlan = channel.unary_unary( + "/spark.connect.SparkConnectService/BatchExecutePlan", + request_serializer=spark_dot_connect_dot_base__pb2.BatchExecutePlanRequest.SerializeToString, + response_deserializer=spark_dot_connect_dot_base__pb2.BatchExecutePlanResponse.FromString, + _registered_method=True, + ) class SparkConnectServiceServicer(object): @@ -192,6 +198,16 @@ def CloneSession(self, request, context): context.set_details("Method not implemented!") raise NotImplementedError("Method not implemented!") + def BatchExecutePlan(self, request, context): + """Execute multiple sequences of plans in batch. + Each sequence executes sequentially in its own thread, all sequences execute in parallel. + Single-plan batches are treated as sequences with one plan. + Returns operation IDs for reattachment. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details("Method not implemented!") + raise NotImplementedError("Method not implemented!") + def add_SparkConnectServiceServicer_to_server(servicer, server): rpc_method_handlers = { @@ -250,6 +266,11 @@ def add_SparkConnectServiceServicer_to_server(servicer, server): request_deserializer=spark_dot_connect_dot_base__pb2.CloneSessionRequest.FromString, response_serializer=spark_dot_connect_dot_base__pb2.CloneSessionResponse.SerializeToString, ), + "BatchExecutePlan": grpc.unary_unary_rpc_method_handler( + servicer.BatchExecutePlan, + request_deserializer=spark_dot_connect_dot_base__pb2.BatchExecutePlanRequest.FromString, + response_serializer=spark_dot_connect_dot_base__pb2.BatchExecutePlanResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( "spark.connect.SparkConnectService", rpc_method_handlers @@ -591,3 +612,33 @@ def CloneSession( metadata, _registered_method=True, ) + + @staticmethod + def BatchExecutePlan( + request, + target, + options=(), + channel_credentials=None, + call_credentials=None, + insecure=False, + compression=None, + wait_for_ready=None, + timeout=None, + metadata=None, + ): + return grpc.experimental.unary_unary( + request, + target, + "/spark.connect.SparkConnectService/BatchExecutePlan", + spark_dot_connect_dot_base__pb2.BatchExecutePlanRequest.SerializeToString, + spark_dot_connect_dot_base__pb2.BatchExecutePlanResponse.FromString, + options, + channel_credentials, + insecure, + call_credentials, + compression, + wait_for_ready, + timeout, + metadata, + _registered_method=True, + ) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBatchExecuteSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBatchExecuteSuite.scala new file mode 100644 index 000000000000..9e2d93408a34 --- /dev/null +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientBatchExecuteSuite.scala @@ -0,0 +1,282 @@ +/* + * 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.client + +import java.util.UUID + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.connect.proto +import org.apache.spark.sql.connect.test.{ConnectFunSuite, RemoteSparkSession} + +class SparkConnectClientBatchExecuteSuite extends ConnectFunSuite with RemoteSparkSession { + + private def buildPlan(query: String): proto.Plan = { + proto.Plan + .newBuilder() + .setRoot( + proto.Relation + .newBuilder() + .setSql(proto.SQL + .newBuilder() + .setQuery(query))) + .build() + } + + test("batchExecute with multiple sequences") { + val seq1 = Seq( + (buildPlan("SELECT * FROM range(100)"), None), + (buildPlan("SELECT * FROM range(200)"), None)) + val seq2 = Seq((buildPlan("SELECT * FROM range(300)"), None)) + + val client = spark.client + val response = client.batchExecute(Seq(seq1, seq2)) + + assert(response.getSequenceResultsCount == 2) + assert(response.getSequenceResults(0).getSuccess, + s"Sequence 1 failed: ${response.getSequenceResults(0).getErrorMessage}") + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 2) + assert(response.getSequenceResults(1).getSuccess, + s"Sequence 2 failed: ${response.getSequenceResults(1).getErrorMessage}") + assert(response.getSequenceResults(1).getQueryOperationIdsCount == 1) + + // Verify operation IDs are valid UUIDs + val seqOpId1 = response.getSequenceResults(0).getSequenceOperationId + val seqOpId2 = response.getSequenceResults(1).getSequenceOperationId + assert(UUID.fromString(seqOpId1) != null) + assert(UUID.fromString(seqOpId2) != null) + + // Give operations time to initialize + Thread.sleep(500) + + // Verify we can reattach to each sequence + val reattachIter1 = client.reattach(seqOpId1) + var hasResponses1 = false + var hasQueryOpId1 = false + while (reattachIter1.hasNext) { + val resp = reattachIter1.next() + hasResponses1 = true + if (resp.hasQueryOperationId) hasQueryOpId1 = true + } + assert(hasResponses1) + assert(hasQueryOpId1) + + val reattachIter2 = client.reattach(seqOpId2) + var hasResponses2 = false + var hasQueryOpId2 = false + while (reattachIter2.hasNext) { + val resp = reattachIter2.next() + hasResponses2 = true + if (resp.hasQueryOperationId) hasQueryOpId2 = true + } + assert(hasResponses2) + assert(hasQueryOpId2) + } + + test("batchExecute with custom sequence operation IDs") { + val seqOpId1 = UUID.randomUUID().toString + val seqOpId2 = UUID.randomUUID().toString + + val seq1 = Seq((buildPlan("SELECT * FROM range(10)"), None)) + val seq2 = Seq((buildPlan("SELECT * FROM range(20)"), None)) + + val client = spark.client + val response = client.batchExecute(Seq(seq1, seq2), Seq(Some(seqOpId1), Some(seqOpId2))) + + assert(response.getSequenceResultsCount == 2) + assert(response.getSequenceResults(0).getSequenceOperationId == seqOpId1) + assert(response.getSequenceResults(1).getSequenceOperationId == seqOpId2) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(1).getSuccess) + } + + test("batchExecute with custom query operation IDs") { + val queryOpId1 = UUID.randomUUID().toString + val queryOpId2 = UUID.randomUUID().toString + + val seq1 = Seq( + (buildPlan("SELECT * FROM range(10)"), Some(queryOpId1)), + (buildPlan("SELECT * FROM range(20)"), Some(queryOpId2))) + + val client = spark.client + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 2) + + val queryOpIds = response.getSequenceResults(0).getQueryOperationIdsList.asScala + assert(queryOpIds.exists(_.getOperationId == queryOpId1)) + assert(queryOpIds.exists(_.getOperationId == queryOpId2)) + assert(queryOpIds.find(_.getOperationId == queryOpId1).get.getQueryIndex == 0) + assert(queryOpIds.find(_.getOperationId == queryOpId2).get.getQueryIndex == 1) + } + + test("batchExecute with invalid sequence operation ID format") { + val client = spark.client + val seq1 = Seq((buildPlan("SELECT 1"), None)) + + val exception = intercept[IllegalArgumentException] { + client.batchExecute(Seq(seq1), Seq(Some("invalid-uuid"))) + } + + assert(exception.getMessage.contains("Invalid sequence operation ID")) + } + + test("batchExecute with invalid query operation ID format") { + val client = spark.client + val seq1 = Seq((buildPlan("SELECT 1"), Some("invalid-uuid"))) + + val exception = intercept[IllegalArgumentException] { + client.batchExecute(Seq(seq1)) + } + + assert(exception.getMessage.contains("Invalid operation ID")) + } + + test("batchExecute with empty sequence list") { + val client = spark.client + val response = client.batchExecute(Seq.empty) + + assert(response.getSequenceResultsCount == 0) + } + + test("batchExecute with empty sequence") { + val client = spark.client + val response = client.batchExecute(Seq(Seq.empty)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 0) + } + + test("batchExecute respects session information") { + val seq1 = Seq((buildPlan("SELECT * FROM range(10)"), None)) + + val client = spark.client + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSessionId == client.sessionId) + assert(response.getServerSideSessionId.nonEmpty) + } + + test("batchExecute with sequential execution verification") { + val seq1 = Seq( + (buildPlan("SELECT * FROM range(100)"), None), + (buildPlan("SELECT * FROM range(200)"), None), + (buildPlan("SELECT * FROM range(300)"), None)) + + val client = spark.client + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 3) + + val seqOpId = response.getSequenceResults(0).getSequenceOperationId + + Thread.sleep(500) + + // Verify we can reattach and consume the sequence + val reattachIter = client.reattach(seqOpId) + var hasResponses = false + while (reattachIter.hasNext) { + reattachIter.next() + hasResponses = true + } + assert(hasResponses) + } + + test("batchExecute with failing query") { + val seq1 = Seq( + (buildPlan("SELECT * FROM range(10)"), None), + (buildPlan("SELECT * FROM range(20)"), None)) + + val client = spark.client + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) // Submission succeeds + + val seqOpId = response.getSequenceResults(0).getSequenceOperationId + + Thread.sleep(500) + + // Should be able to reattach and consume results + val reattachIter = client.reattach(seqOpId) + var hasResponses = false + while (reattachIter.hasNext) { + reattachIter.next() + hasResponses = true + } + assert(hasResponses) + } + + test("batchExecute with multiple sequences executes in parallel") { + // Create multiple sequences with long-running queries + val sequences = (1 to 3).map { i => + Seq((buildPlan(s"SELECT * FROM range(1000)"), None)) + } + + val startTime = System.currentTimeMillis() + val client = spark.client + val response = client.batchExecute(sequences) + val submitTime = System.currentTimeMillis() - startTime + + // Submission should be fast (not waiting for execution) + assert(submitTime < 5000) + + assert(response.getSequenceResultsCount == 3) + response.getSequenceResultsList.asScala.foreach { result => + assert(result.getSuccess) + assert(result.getQueryOperationIdsCount == 1) + } + } + + test("batchExecute with duplicate sequence operation ID") { + val client = spark.client + val seqOpId = UUID.randomUUID().toString + val seq1 = Seq((buildPlan("SELECT * FROM range(10)"), None)) + + // First submission should succeed + val response1 = client.batchExecute(Seq(seq1), Seq(Some(seqOpId))) + assert(response1.getSequenceResults(0).getSuccess) + + // Give it time to start + Thread.sleep(100) + + // Second submission with same ID should report failure + val seq2 = Seq((buildPlan("SELECT * FROM range(20)"), None)) + val response2 = client.batchExecute(Seq(seq2), Seq(Some(seqOpId))) + assert(!response2.getSequenceResults(0).getSuccess) + assert(response2.getSequenceResults(0).hasErrorMessage) + } + + test("batchExecute with mix of valid and invalid UUIDs") { + val client = spark.client + val validUUID = UUID.randomUUID().toString + val seq1 = + Seq((buildPlan("SELECT 1"), Some(validUUID)), (buildPlan("SELECT 2"), Some("invalid"))) + + val exception = intercept[IllegalArgumentException] { + client.batchExecute(Seq(seq1)) + } + + assert(exception.getMessage.contains("Invalid operation ID")) + } +} 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 6e1029bf0a6a..133efd20e6a5 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -347,7 +347,7 @@ message ExecutePlanRequest { // The response of a query, can be one or more for each request. Responses belonging to the // same input query, carry the same `session_id`. -// Next ID: 17 +// Next ID: 25 message ExecutePlanResponse { string session_id = 1; // Server-side generated idempotency key that the client can use to assert that the server side @@ -364,6 +364,10 @@ message ExecutePlanResponse { // The id is an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` string response_id = 13; + // (Optional) For sequence executions: identifies which query in the sequence this response belongs to. + // This is set when the execution is part of a BatchExecutePlan sequence. + optional string query_operation_id = 24; + // Union type for the different response messages. oneof response_type { ArrowBatch arrow_batch = 2; @@ -1200,6 +1204,87 @@ message CloneSessionResponse { string new_server_side_session_id = 4; } +// Request to execute multiple sequences of plans in batch. +// Each sequence executes sequentially in its own thread, all sequences execute in parallel. +// Single-plan batches are treated as sequences with one plan. +message BatchExecutePlanRequest { + // (Required) The session_id for which this batch execute is targeting. + string session_id = 1; + + // (Optional) Client-side observed server-side session ID. + optional string client_observed_server_side_session_id = 2; + + // (Required) User context + UserContext user_context = 3; + + // (Optional) An optional client-type identifier (e.g., "spark-sql", "pyspark"). + optional string client_type = 4; + + // (Required) List of plan sequences to execute. Each sequence runs in parallel. + repeated PlanSequence plan_sequences = 5; + + // A sequence of plans that execute sequentially + message PlanSequence { + // (Optional) Operation ID for the entire sequence. + // If not provided, a UUID will be generated by the server. + // The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` + optional string sequence_operation_id = 1; + + // (Required) Plans to execute sequentially within this sequence + repeated PlanExecution plan_executions = 2; + } + + // A single plan execution within a sequence + message PlanExecution { + // (Required) The logical plan to execute. + Plan plan = 1; + + // (Optional) Operation ID for this individual query. + // If not provided, a UUID will be generated by the server. + // The id must be an UUID string of the format `00112233-4455-6677-8899-aabbccddeeff` + optional string operation_id = 2; + + // (Optional) Tags to be assigned to the operations. + repeated string tags = 3; + } +} + +// Response for batch execute plan request. +message BatchExecutePlanResponse { + // Session id in which the batch was executed. + string session_id = 1; + + // Server-side generated idempotency key for the session. + string server_side_session_id = 2; + + // Results for each sequence in the same order as the request + repeated SequenceResult sequence_results = 3; + + // Result for a single sequence execution + message SequenceResult { + // (Required) Operation ID for the entire sequence (reattach to this) + string sequence_operation_id = 1; + + // (Required) Whether the sequence was successfully submitted + bool success = 2; + + // (Optional) Error message if submission failed + optional string error_message = 3; + + // (Required) Operation IDs for individual queries within the sequence + repeated QueryOperationId query_operation_ids = 4; + } + + // Query operation ID with its index in the sequence + message QueryOperationId { + // (Required) Operation ID for this individual query + string operation_id = 1; + + // (Required) Index of this query in the sequence (0-based) + int32 query_index = 2; + } +} + // Main interface for the SparkConnect service. service SparkConnectService { @@ -1254,4 +1339,10 @@ service SparkConnectService { // The request can optionally specify a custom session ID for the cloned session (must be // a valid UUID). If not provided, a new UUID will be generated automatically. rpc CloneSession(CloneSessionRequest) returns (CloneSessionResponse) {} + + // Execute multiple sequences of plans in batch. + // Each sequence executes sequentially in its own thread, all sequences execute in parallel. + // Single-plan batches are treated as sequences with one plan. + // Returns operation IDs for reattachment. + rpc BatchExecutePlan(BatchExecutePlanRequest) returns (BatchExecutePlanResponse) {} } diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala index 913f068fcf34..a0fad70243aa 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala @@ -80,6 +80,37 @@ private[connect] class CustomSparkConnectBlockingStub( } } + def batchExecutePlan(request: BatchExecutePlanRequest): BatchExecutePlanResponse = { + grpcExceptionConverter.convert( + request.getSessionId, + request.getUserContext, + request.getClientType) { + retryHandler.retry { + stubState.responseValidator.verifyResponse { + stub.batchExecutePlan(request) + } + } + } + } + + def reattachExecute(request: ReattachExecuteRequest): CloseableIterator[ExecutePlanResponse] = { + grpcExceptionConverter.convert( + request.getSessionId, + request.getUserContext, + request.getClientType) { + grpcExceptionConverter.convertIterator[ExecutePlanResponse]( + request.getSessionId, + request.getUserContext, + request.getClientType, + retryHandler.RetryIterator[ReattachExecuteRequest, ExecutePlanResponse]( + request, + r => { + stubState.responseValidator.wrapIterator( + CloseableIterator(stub.reattachExecute(r).asScala)) + })) + } + } + def config(request: ConfigRequest): ConfigResponse = { grpcExceptionConverter.convert( request.getSessionId, diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index fa32eba91eb2..9b7390c91148 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -153,6 +153,95 @@ private[sql] class SparkConnectClient( } } + /** + * Execute multiple sequences of plans in batch. + * + * Each sequence executes sequentially, all sequences execute in parallel. Single-plan batches + * are treated as sequences containing one plan. Returns operation IDs for reattachment. + * + * @param planSequences + * Sequences of (plan, optional operation ID) tuples to execute + * @param sequenceOperationIds + * Optional operation IDs for each sequence + * @return + * BatchExecutePlanResponse containing sequence operation IDs and query operation IDs + */ + def batchExecute( + planSequences: Seq[Seq[(proto.Plan, Option[String])]], + sequenceOperationIds: Seq[Option[String]] = Seq.empty): proto.BatchExecutePlanResponse = { + artifactManager.uploadAllClassFileArtifacts() + + val requestBuilder = proto.BatchExecutePlanRequest + .newBuilder() + .setUserContext(userContext) + .setSessionId(sessionId) + .setClientType(userAgent) + + serverSideSessionId.foreach(session => + requestBuilder.setClientObservedServerSideSessionId(session)) + + planSequences.zipWithIndex.foreach { case (sequence, idx) => + val sequenceBuilder = proto.BatchExecutePlanRequest.PlanSequence + .newBuilder() + + // Set sequence operation ID if provided + sequenceOperationIds.lift(idx).flatten.foreach { seqOpId => + require( + isValidUUID(seqOpId), + s"Invalid sequence operation ID: $seqOpId. The id must be an UUID string of " + + "the format `00112233-4455-6677-8899-aabbccddeeff`") + sequenceBuilder.setSequenceOperationId(seqOpId) + } + + // Add plans to sequence + sequence.foreach { case (plan, opId) => + val planExecBuilder = proto.BatchExecutePlanRequest.PlanExecution + .newBuilder() + .setPlan(plan) + opId.foreach { id => + require( + isValidUUID(id), + s"Invalid operation ID: $id. The id must be an UUID string of " + + "the format `00112233-4455-6677-8899-aabbccddeeff`") + planExecBuilder.setOperationId(id) + } + sequenceBuilder.addPlanExecutions(planExecBuilder) + } + + requestBuilder.addPlanSequences(sequenceBuilder) + } + + bstub.batchExecutePlan(requestBuilder.build()) + } + + /** + * Reattach to an existing operation by operation ID and consume all responses. This method will + * block until the operation completes. + * + * @param operationId + * The operation ID to reattach to + * @return + * An iterator of ExecutePlanResponse messages + */ + def reattach(operationId: String): CloseableIterator[proto.ExecutePlanResponse] = { + require( + isValidUUID(operationId), + s"Invalid operationId: $operationId. The id must be an UUID string of " + + "the format `00112233-4455-6677-8899-aabbccddeeff`") + + val requestBuilder = proto.ReattachExecuteRequest + .newBuilder() + .setSessionId(sessionId) + .setUserContext(userContext) + .setOperationId(operationId) + .setClientType(userAgent) + + serverSideSessionId.foreach(session => + requestBuilder.setClientObservedServerSideSessionId(session)) + + bstub.reattachExecute(requestBuilder.build()) + } + /** * Dispatch the [[proto.ConfigRequest]] to the Spark Connect server. * @return diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SequenceExecuteThreadRunner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SequenceExecuteThreadRunner.scala new file mode 100644 index 000000000000..c2479083c15f --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/execution/SequenceExecuteThreadRunner.scala @@ -0,0 +1,251 @@ +/* + * 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.execution + +import org.apache.spark.connect.proto +import org.apache.spark.internal.{Logging, LogKeys} +import org.apache.spark.sql.classic.Dataset +import org.apache.spark.sql.connect.common.DataTypeProtoConverter +import org.apache.spark.sql.connect.planner.SparkConnectPlanner +import org.apache.spark.sql.connect.service.{ExecuteHolder, ExecuteSessionTag, SparkConnectService} +import org.apache.spark.sql.connect.utils.ErrorUtils +import org.apache.spark.util.Utils + +/** + * Custom execution runner for batch list execution sequences. Executes a sequence of plans + * sequentially, tagging each response with its query operation ID. + * + * @param executeHolder + * the ExecuteHolder for this sequence execution + * @param plansWithIds + * sequence of (Plan, queryOperationId, tags) to execute + */ +private[connect] class SequenceExecuteThreadRunner( + executeHolder: ExecuteHolder, + plansWithIds: Seq[(proto.Plan, String, Seq[String])]) + extends ExecuteThreadRunner(executeHolder) + with Logging { + + private val sequenceExecutionThread: SequenceExecutionThread = + new SequenceExecutionThread() + + @volatile private var started = false + @volatile private var interrupted = false + + override def start(): Unit = { + started = true + sequenceExecutionThread.start() + } + + override def interrupt(): Boolean = { + if (started && !interrupted) { + interrupted = true + sequenceExecutionThread.interrupt() + true + } else { + false + } + } + + private def execute(): Unit = { + var success = false + try { + try { + executeSequence() + success = true + } catch { + case e: Throwable => + logDebug(log"Exception in execute: ${MDC(LogKeys.EXCEPTION, e)}") + executeHolder.sessionHolder.session.sparkContext.cancelJobsWithTag( + executeHolder.jobTag, + s"A job with the same tag ${executeHolder.jobTag} has failed.") + throw e + } finally { + executeHolder.sessionHolder.session.sparkContext.removeJobTag(executeHolder.jobTag) + SparkConnectService.executionListener.foreach(_.removeJobTag(executeHolder.jobTag)) + executeHolder.sparkSessionTags.foreach { tag => + executeHolder.sessionHolder.session.sparkContext.removeJobTag( + ExecuteSessionTag( + executeHolder.sessionHolder.userId, + executeHolder.sessionHolder.sessionId, + tag)) + } + } + } catch { + case e: Throwable => + ErrorUtils.handleError( + "execute", + executeHolder.responseObserver, + executeHolder.sessionHolder.userId, + executeHolder.sessionHolder.sessionId, + Some(executeHolder.eventsManager), + false)(e) + } finally { + // Ensure observer is always completed, even if something went wrong + if (!success && !executeHolder.responseObserver.completed()) { + logWarning( + log"Sequence ${MDC(LogKeys.EXECUTE_KEY, executeHolder.operationId)} " + + log"finished without completing observer, completing now") + try { + executeHolder.responseObserver.onCompleted() + } catch { + case _: IllegalStateException => // Already completed, ignore + } + } + } + } + + private def executeSequence(): Unit = { + executeHolder.sessionHolder.withSession { session => + // Set tag for query cancellation + session.sparkContext.addJobTag(executeHolder.jobTag) + SparkConnectService.executionListener.foreach(_.registerJobTag(executeHolder.jobTag)) + + // Also set all user defined tags as Spark Job tags + executeHolder.sparkSessionTags.foreach { tag => + session.sparkContext.addJobTag( + ExecuteSessionTag( + executeHolder.sessionHolder.userId, + executeHolder.sessionHolder.sessionId, + tag)) + } + + val debugString = + s"BatchListExecute sequence with ${plansWithIds.size} queries" + session.sparkContext.setJobDescription( + s"Spark Connect - ${Utils.abbreviate(debugString, 128)}") + session.sparkContext.setInterruptOnCancel(true) + + session.sparkContext.setLocalProperty( + "callSite.short", + s"Spark Connect - ${Utils.abbreviate(debugString, 128)}") + session.sparkContext.setLocalProperty("callSite.long", Utils.abbreviate(debugString, 2048)) + + try { + // Execute each plan in the sequence + plansWithIds.foreach { case (plan, queryOperationId, tags) => + logInfo(s"Executing query $queryOperationId in sequence ${executeHolder.operationId}") + + // Create a wrapper observer that adds query_operation_id to responses + val wrappedObserver = + new QueryTaggingResponseObserver(executeHolder.responseObserver, queryOperationId) + + // Execute the plan based on its type + plan.getOpTypeCase match { + case proto.Plan.OpTypeCase.ROOT => + val planner = new SparkConnectPlanner(executeHolder) + val tracker = executeHolder.eventsManager.createQueryPlanningTracker() + val dataframe = Dataset.ofRows( + session, + planner.transformRelation(plan.getRoot, cachePlan = true), + tracker) + + // Send schema + wrappedObserver.onNext(createSchemaResponse(dataframe.schema)) + + // Process as arrow batches + val planExecution = new SparkConnectPlanExecution(executeHolder) + planExecution.processAsArrowBatches(dataframe, wrappedObserver, executeHolder) + + case proto.Plan.OpTypeCase.COMMAND => + val planner = new SparkConnectPlanner(executeHolder) + planner.transformCommand(plan.getCommand) match { + case Some(transformer) => + val tracker = executeHolder.eventsManager.createQueryPlanningTracker() + val qe = new org.apache.spark.sql.execution.QueryExecution( + session, + transformer(tracker), + tracker) + qe.assertCommandExecuted() + case None => + planner.process(plan.getCommand, wrappedObserver) + } + + case other => + throw new IllegalArgumentException(s"Unsupported plan type: $other") + } + + logInfo(s"Completed query $queryOperationId in sequence ${executeHolder.operationId}") + } + + // All queries in sequence completed successfully + executeHolder.eventsManager.postFinished() + + // Send ResultComplete and complete the observer + if (!executeHolder.responseObserver.completed()) { + executeHolder.responseObserver.onNext(createResultComplete()) + executeHolder.responseObserver.onCompleted() + } + + } catch { + case e: Throwable => + logError( + log"Sequence ${MDC(LogKeys.EXECUTE_KEY, executeHolder.operationId)} failed: " + + log"${MDC(LogKeys.EXCEPTION, e)}") + throw e + } + } + } + + private def createSchemaResponse( + schema: org.apache.spark.sql.types.StructType): proto.ExecutePlanResponse = { + proto.ExecutePlanResponse + .newBuilder() + .setSessionId(executeHolder.request.getSessionId) + .setServerSideSessionId(executeHolder.sessionHolder.serverSessionId) + .setSchema(DataTypeProtoConverter.toConnectProtoType(schema)) + .build() + } + + private def createResultComplete(): proto.ExecutePlanResponse = { + proto.ExecutePlanResponse + .newBuilder() + .setResultComplete(proto.ExecutePlanResponse.ResultComplete.newBuilder().build()) + .build() + } + + private class SequenceExecutionThread() + extends Thread(s"SparkConnectSequenceExecuteThread_opId=${executeHolder.operationId}") { + override def run(): Unit = execute() + } +} + +/** + * Wrapper observer that adds query_operation_id to each response. + */ +private class QueryTaggingResponseObserver( + underlying: ExecuteResponseObserver[proto.ExecutePlanResponse], + queryOperationId: String) + extends io.grpc.stub.StreamObserver[proto.ExecutePlanResponse] { + + override def onNext(response: proto.ExecutePlanResponse): Unit = { + // Add query_operation_id to the response + val modifiedResponse = response.toBuilder + .setQueryOperationId(queryOperationId) + .build() + underlying.onNext(modifiedResponse) + } + + override def onError(throwable: Throwable): Unit = { + underlying.onError(throwable) + } + + override def onCompleted(): Unit = { + // Don't complete the underlying observer - the sequence may have more queries + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala index 42574b1f8d43..92b186b18c12 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala @@ -115,7 +115,14 @@ private[connect] class ExecuteHolder( ExecuteHolder.collectAllObservationAndPlanIds(request.getPlan).toMap } - private val runner: ExecuteThreadRunner = new ExecuteThreadRunner(this) + private var runner: ExecuteThreadRunner = new ExecuteThreadRunner(this) + + /** + * Set a custom runner for this execution. Must be called before start(). + */ + private[service] def setRunner(customRunner: ExecuteThreadRunner): Unit = { + runner = customRunner + } /** System.nanoTime when this ExecuteHolder was created. */ val creationTimeNs = System.nanoTime() diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecutePlanHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecutePlanHandler.scala new file mode 100644 index 000000000000..2486360597d4 --- /dev/null +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecutePlanHandler.scala @@ -0,0 +1,250 @@ +/* + * 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.service + +import java.util.UUID + +import scala.collection.mutable.ArrayBuffer +import scala.jdk.CollectionConverters._ +import scala.util.control.NonFatal + +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.execution.SequenceExecuteThreadRunner + +/** + * Handler for BatchExecutePlan RPC. + * + * This handler submits multiple sequences of execution plans, where each sequence executes + * sequentially in its own thread, and all sequences execute in parallel. Each sequence is + * reattachable as a single operation. Individual queries within a sequence are not separately + * reattachable. + * + * Single-plan batches are treated as sequences containing one plan. + * + * The handler returns submission status and operation IDs for each sequence and its queries. + */ +class SparkConnectBatchExecutePlanHandler( + responseObserver: StreamObserver[proto.BatchExecutePlanResponse]) + extends Logging { + + def handle(request: proto.BatchExecutePlanRequest): Unit = { + logInfo(s"BatchExecutePlan handler called with ${request.getPlanSequencesCount} sequences") + + val previousSessionId = request.hasClientObservedServerSideSessionId match { + case true => Some(request.getClientObservedServerSideSessionId) + case false => None + } + + val sessionHolder = SparkConnectService.getOrCreateIsolatedSession( + request.getUserContext.getUserId, + request.getSessionId, + previousSessionId) + + val sequenceResults = new ArrayBuffer[proto.BatchExecutePlanResponse.SequenceResult]() + + // Process each sequence + for (planSequence <- request.getPlanSequencesList.asScala) { + try { + // Generate or validate sequence operation ID + val sequenceOperationId = if (planSequence.hasSequenceOperationId) { + val provided = planSequence.getSequenceOperationId + logInfo(s"Validating provided sequence operation ID: $provided") + try { + UUID.fromString(provided) + logInfo(s"Sequence operation ID is valid: $provided") + provided + } catch { + case e: IllegalArgumentException => + logError(s"Invalid sequence operation ID format: $provided", e) + throw new SparkSQLException( + errorClass = "INVALID_HANDLE.FORMAT", + messageParameters = Map("handle" -> provided)) + } + } else { + val generated = UUID.randomUUID().toString + logInfo(s"Generated new sequence operation ID: $generated") + generated + } + + // Collect query operation IDs and validate them + val queryOperationIds = + new ArrayBuffer[proto.BatchExecutePlanResponse.QueryOperationId]() + val plansWithIds = new ArrayBuffer[(proto.Plan, String, Seq[String])]() + + planSequence.getPlanExecutionsList.asScala.zipWithIndex.foreach { + case (planExecution, index) => + val queryOperationId = if (planExecution.hasOperationId) { + val provided = planExecution.getOperationId + logInfo(s"Validating provided query operation ID at index $index: $provided") + try { + UUID.fromString(provided) + logInfo(s"Query operation ID is valid: $provided") + provided + } catch { + case e: IllegalArgumentException => + logError(s"Invalid query operation ID format at index $index: $provided", e) + throw new SparkSQLException( + errorClass = "INVALID_HANDLE.FORMAT", + messageParameters = Map("handle" -> provided)) + } + } else { + val generated = UUID.randomUUID().toString + logInfo(s"Generated new query operation ID at index $index: $generated") + generated + } + + queryOperationIds += proto.BatchExecutePlanResponse.QueryOperationId + .newBuilder() + .setOperationId(queryOperationId) + .setQueryIndex(index) + .build() + + plansWithIds += (( + planExecution.getPlan, + queryOperationId, + planExecution.getTagsList.asScala.toSeq)) + } + + // Create a special ExecutePlanRequest that will be handled by SequenceExecuteThreadRunner + logInfo(s"Creating ExecutePlanRequest with sequence operation ID: $sequenceOperationId") + val executePlanRequestBuilder = proto.ExecutePlanRequest + .newBuilder() + .setUserContext(request.getUserContext) + .setSessionId(request.getSessionId) + .setOperationId(sequenceOperationId) + + if (request.hasClientType) { + executePlanRequestBuilder.setClientType(request.getClientType) + } + + if (request.hasClientObservedServerSideSessionId) { + executePlanRequestBuilder.setClientObservedServerSideSessionId( + request.getClientObservedServerSideSessionId) + } + + // Mark as reattachable + val reattachOptions = proto.ReattachOptions.newBuilder().setReattachable(true).build() + val requestOption = proto.ExecutePlanRequest.RequestOption + .newBuilder() + .setReattachOptions(reattachOptions) + .build() + executePlanRequestBuilder.addRequestOptions(requestOption) + + // Set the first plan from the sequence as the representative plan for the ExecuteKey + // The actual execution will use all plans from plansWithIds + val firstPlan = if (plansWithIds.nonEmpty) { + plansWithIds.head._1 + } else { + // Empty sequence - create a minimal valid plan + proto.Plan + .newBuilder() + .setRoot( + proto.Relation.newBuilder().setLocalRelation(proto.LocalRelation.newBuilder())) + .build() + } + executePlanRequestBuilder.setPlan(firstPlan) + + val executePlanRequest = executePlanRequestBuilder.build() + logInfo( + s"Built ExecutePlanRequest: operationId=${executePlanRequest.getOperationId}, " + + s"hasOperationId=${executePlanRequest.hasOperationId}, " + + s"sessionId=${executePlanRequest.getSessionId}, " + + s"planType=${executePlanRequest.getPlan.getOpTypeCase}") + val executeKey = ExecuteKey(executePlanRequest, sessionHolder) + logInfo( + s"Created ExecuteKey: userId=${executeKey.userId}, sessionId=${executeKey.sessionId}, " + + s"operationId=${executeKey.operationId}") + + logInfo(s"Checking for duplicate operation ID: $sequenceOperationId") + + // Check if operation already exists + if (SparkConnectService.executionManager.getExecuteHolder(executeKey).isDefined) { + logError(s"Operation ID already exists: $sequenceOperationId") + throw new SparkSQLException( + errorClass = "INVALID_HANDLE.OPERATION_ALREADY_EXISTS", + messageParameters = Map("handle" -> sequenceOperationId)) + } + + logInfo(s"Creating execute holder for sequence: $sequenceOperationId") + // Create the execute holder with custom runner + val executeHolder = SparkConnectService.executionManager.createExecuteHolder( + executeKey, + executePlanRequest, + sessionHolder) + + logInfo(s"Setting custom sequence runner for: $sequenceOperationId") + // Replace the standard runner with our sequence runner + val sequenceRunner = new SequenceExecuteThreadRunner(executeHolder, plansWithIds.toSeq) + executeHolder.setRunner(sequenceRunner) + + logInfo(s"Starting execution for sequence: $sequenceOperationId") + // Start the execution + executeHolder.eventsManager.postStarted() + executeHolder.start() + executeHolder.afterInitialRPC() + + logInfo(s"Successfully started sequence: $sequenceOperationId") + + // Build success result + sequenceResults += proto.BatchExecutePlanResponse.SequenceResult + .newBuilder() + .setSequenceOperationId(sequenceOperationId) + .setSuccess(true) + .addAllQueryOperationIds(queryOperationIds.asJava) + .build() + + } catch { + case NonFatal(e) => + // Submission failed for this sequence + val sequenceOperationId = if (planSequence.hasSequenceOperationId) { + planSequence.getSequenceOperationId + } else { + UUID.randomUUID().toString + } + + val errorMessage = s"${e.getClass.getSimpleName}: ${e.getMessage}" + + logError(s"Failed to submit sequence $sequenceOperationId: $errorMessage", e) + + sequenceResults += proto.BatchExecutePlanResponse.SequenceResult + .newBuilder() + .setSequenceOperationId(sequenceOperationId) + .setSuccess(false) + .setErrorMessage(errorMessage) + .build() + } + } + + logInfo(s"BatchExecutePlan completed, sending response with ${sequenceResults.size} results") + + // Build and send the response + val response = proto.BatchExecutePlanResponse + .newBuilder() + .setSessionId(request.getSessionId) + .setServerSideSessionId(sessionHolder.serverSessionId) + .addAllSequenceResults(sequenceResults.asJava) + .build() + + responseObserver.onNext(response) + responseObserver.onCompleted() + } +} diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 13ce2d64256b..287bdac337e7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -105,6 +105,32 @@ class SparkConnectService(debug: Boolean) extends AsyncService with BindableServ } } + /** + * Execute multiple sequences of plans in batch. + * + * Each sequence executes sequentially in its own thread, and all sequences execute in parallel. + * Single-plan batches are treated as sequences containing one plan. Returns operation IDs for + * reattachment. + * + * The batch execution is delegated to the [[SparkConnectBatchExecutePlanHandler]]. + * + * @param request + * @param responseObserver + */ + override def batchExecutePlan( + request: proto.BatchExecutePlanRequest, + responseObserver: StreamObserver[proto.BatchExecutePlanResponse]): Unit = { + try { + new SparkConnectBatchExecutePlanHandler(responseObserver).handle(request) + } catch { + ErrorUtils.handleError( + "batchExecute", + observer = responseObserver, + userId = request.getUserContext.getUserId, + sessionId = request.getSessionId) + } + } + /** * This is the main entry method for Spark Connect and all calls to update or fetch * configuration.. diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecuteSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecuteSuite.scala new file mode 100644 index 000000000000..0ee6559af069 --- /dev/null +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectBatchExecuteSuite.scala @@ -0,0 +1,293 @@ +/* + * 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.service + +import java.util.UUID + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.connect.proto +import org.apache.spark.sql.connect.SparkConnectServerTest + +class SparkConnectBatchExecuteSuite extends SparkConnectServerTest { + + private def createPlan(query: String): proto.Plan = { + proto.Plan + .newBuilder() + .setRoot( + proto.Relation + .newBuilder() + .setSql(proto.SQL + .newBuilder() + .setQuery(query))) + .build() + } + + test("batch execute with multiple sequences") { + withClient { client => + val seq1 = Seq((createPlan("SELECT 1"), None), (createPlan("SELECT 2"), None)) + val seq2 = Seq((createPlan("SELECT 3"), None), (createPlan("SELECT 4"), None)) + + val response = client.batchExecute(Seq(seq1, seq2)) + + assert(response.getSequenceResultsCount == 2) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 2) + assert(response.getSequenceResults(1).getSuccess) + assert(response.getSequenceResults(1).getQueryOperationIdsCount == 2) + + // Verify we can reattach to each sequence + val seqOpId1 = response.getSequenceResults(0).getSequenceOperationId + val seqOpId2 = response.getSequenceResults(1).getSequenceOperationId + + Thread.sleep(100) // Give sequences time to initialize + + // Verify we can reattach to each sequence and they complete + val reattachIter1 = client.reattach(seqOpId1) + var hasQueryOpId1 = false + var hasResultComplete1 = false + while (reattachIter1.hasNext) { + val resp = reattachIter1.next() + if (resp.hasQueryOperationId) hasQueryOpId1 = true + if (resp.hasResultComplete) hasResultComplete1 = true + } + assert(hasQueryOpId1) + assert(hasResultComplete1) + + val reattachIter2 = client.reattach(seqOpId2) + var hasQueryOpId2 = false + var hasResultComplete2 = false + while (reattachIter2.hasNext) { + val resp = reattachIter2.next() + if (resp.hasQueryOperationId) hasQueryOpId2 = true + if (resp.hasResultComplete) hasResultComplete2 = true + } + assert(hasQueryOpId2) + assert(hasResultComplete2) + } + } + + test("batch execute with custom sequence operation IDs") { + withClient { client => + val seqOpId1 = UUID.randomUUID().toString + val seqOpId2 = UUID.randomUUID().toString + + val seq1 = Seq((createPlan("SELECT 1"), None)) + val seq2 = Seq((createPlan("SELECT 2"), None)) + + val response = client.batchExecute(Seq(seq1, seq2), Seq(Some(seqOpId1), Some(seqOpId2))) + + assert(response.getSequenceResultsCount == 2) + assert(response.getSequenceResults(0).getSequenceOperationId == seqOpId1) + assert(response.getSequenceResults(1).getSequenceOperationId == seqOpId2) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(1).getSuccess) + } + } + + test("batch execute with custom query operation IDs") { + withClient { client => + val queryOpId1 = UUID.randomUUID().toString + val queryOpId2 = UUID.randomUUID().toString + + val seq1 = Seq( + (createPlan("SELECT 1"), Some(queryOpId1)), + (createPlan("SELECT 2"), Some(queryOpId2))) + + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 2) + + val queryOpIds = response.getSequenceResults(0).getQueryOperationIdsList.asScala + assert(queryOpIds.exists(_.getOperationId == queryOpId1)) + assert(queryOpIds.exists(_.getOperationId == queryOpId2)) + } + } + + test("batch execute with invalid sequence operation ID format") { + withClient { client => + val seq1 = Seq((createPlan("SELECT 1"), None)) + + // Client-side validation should throw IllegalArgumentException + val ex = intercept[IllegalArgumentException] { + client.batchExecute(Seq(seq1), Seq(Some("invalid-uuid"))) + } + assert(ex.getMessage.contains("Invalid sequence operation ID")) + } + } + + test("batch execute with duplicate sequence operation ID") { + withClient { client => + val seqOpId = UUID.randomUUID().toString + + val seq1 = Seq((createPlan("SELECT 1"), None)) + val seq2 = Seq((createPlan("SELECT 2"), None)) + + // First submission should succeed + val response1 = client.batchExecute(Seq(seq1), Seq(Some(seqOpId))) + assert(response1.getSequenceResults(0).getSuccess) + + // Second submission with same ID should fail + val response2 = client.batchExecute(Seq(seq2), Seq(Some(seqOpId))) + assert(!response2.getSequenceResults(0).getSuccess) + assert(response2.getSequenceResults(0).getErrorMessage.contains("OPERATION_ALREADY_EXISTS")) + } + } + + test("batch execute with failing query stops sequence") { + withClient { client => + val seq1 = Seq( + (createPlan("SELECT 1"), None), + (createPlan("SELECT * FROM non_existent_table"), None), + (createPlan("SELECT 2"), None) + ) // This should not execute + + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) // Submission succeeds + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 3) + + val seqOpId = response.getSequenceResults(0).getSequenceOperationId + + Thread.sleep(100) // Give sequence time to start + + // Reattach to see the error + val exception = intercept[Exception] { + val reattachIter = client.reattach(seqOpId) + while (reattachIter.hasNext) { + reattachIter.next() + } + } + + assert( + exception.getMessage.contains("TABLE_OR_VIEW_NOT_FOUND") || + exception.getMessage.contains("non_existent_table")) + } + } + + test("batch execute with empty sequence list") { + withClient { client => + val response = client.batchExecute(Seq.empty) + + assert(response.getSequenceResultsCount == 0) + } + } + + test("batch execute with single query in sequence") { + withClient { client => + val seq1 = Seq((createPlan("SELECT 42"), None)) + + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 1) + + val seqOpId = response.getSequenceResults(0).getSequenceOperationId + + Thread.sleep(100) + + val reattachIter = client.reattach(seqOpId) + var hasQueryOpId = false + var hasArrowBatch = false + while (reattachIter.hasNext) { + val resp = reattachIter.next() + if (resp.hasQueryOperationId) hasQueryOpId = true + if (resp.hasArrowBatch) hasArrowBatch = true + } + assert(hasQueryOpId) + assert(hasArrowBatch) + } + } + + test("batch execute respects session ID") { + withClient { client => + val seq1 = Seq((createPlan("SELECT 1"), None)) + + val response = client.batchExecute(Seq(seq1)) + + assert(response.getSessionId == client.sessionId) + assert(response.getServerSideSessionId.nonEmpty) + } + } + + test("batch execute with multiple queries in single sequence") { + withClient { client => + val queries = (1 to 5).map(i => (createPlan(s"SELECT $i"), None)) + + val response = client.batchExecute(Seq(queries)) + + assert(response.getSequenceResultsCount == 1) + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(0).getQueryOperationIdsCount == 5) + + // Verify query indices are correct + val queryOpIds = response.getSequenceResults(0).getQueryOperationIdsList.asScala + queryOpIds.zipWithIndex.foreach { case (qid, idx) => + assert(qid.getQueryIndex == idx) + } + } + } + + test("batch execute with mixed success and failure sequences") { + withClient { client => + val seq1 = Seq((createPlan("SELECT 1"), None)) // Success + val seq2 = Seq((createPlan("INVALID SQL"), None)) // Will fail during execution + val seq3 = Seq((createPlan("SELECT 3"), None)) // Success + + val response = client.batchExecute(Seq(seq1, seq2, seq3)) + + assert(response.getSequenceResultsCount == 3) + // All should submit successfully + assert(response.getSequenceResults(0).getSuccess) + assert(response.getSequenceResults(1).getSuccess) + assert(response.getSequenceResults(2).getSuccess) + + Thread.sleep(100) + + // seq1 should complete successfully + val reattachIter1 = client.reattach(response.getSequenceResults(0).getSequenceOperationId) + var hasResponses1 = false + while (reattachIter1.hasNext) { + reattachIter1.next() + hasResponses1 = true + } + assert(hasResponses1) + + // seq2 should fail during execution (reattach will throw) + intercept[Exception] { + val reattachIter2 = client.reattach(response.getSequenceResults(1).getSequenceOperationId) + while (reattachIter2.hasNext) { + reattachIter2.next() + } + } + + // seq3 should complete successfully + val reattachIter3 = client.reattach(response.getSequenceResults(2).getSequenceOperationId) + var hasResponses3 = false + while (reattachIter3.hasNext) { + reattachIter3.next() + hasResponses3 = true + } + assert(hasResponses3) + } + } +}