diff --git a/connector/connect/common/src/main/protobuf/spark/connect/base.proto b/connector/connect/common/src/main/protobuf/spark/connect/base.proto index 77dda277602ab..a14efce780e40 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/base.proto @@ -94,6 +94,7 @@ message AnalyzePlanRequest { Persist persist = 14; Unpersist unpersist = 15; GetStorageLevel get_storage_level = 16; + Checkpoint checkpoint = 18; } message Schema { @@ -199,6 +200,17 @@ message AnalyzePlanRequest { // (Required) The logical plan to get the storage level. Relation relation = 1; } + + message Checkpoint { + // (Required) The logical plan to checkpoint. + Relation relation = 1; + + // (Optional) Is it localCheckpoint + optional bool local = 2; + + // (Optional) Is it eager + optional bool eager = 3; + } } // Response to performing analysis of the query. Contains relevant metadata to be able to @@ -224,6 +236,7 @@ message AnalyzePlanResponse { Persist persist = 12; Unpersist unpersist = 13; GetStorageLevel get_storage_level = 14; + Checkpoint checkpoint = 16; } message Schema { @@ -275,6 +288,11 @@ message AnalyzePlanResponse { // (Required) The StorageLevel as a result of get_storage_level request. StorageLevel storage_level = 1; } + + message Checkpoint { + // (Required) The logical plan checkpointed. + CachedRemoteRelation relation = 1; + } } // A request to be executed by the service. diff --git a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto index acff0a2089e95..9f561e7e893c9 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -45,6 +45,7 @@ message Command { StreamingQueryListenerBusCommand streaming_query_listener_bus_command = 11; CommonInlineUserDefinedDataSource register_data_source = 12; CreateResourceProfileCommand create_resource_profile_command = 13; + RemoveCachedRemoteRelationCommand remove_cached_remote_relation_command = 14; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // Commands they can add them here. During the planning the correct resolution is done. @@ -484,3 +485,9 @@ message CreateResourceProfileCommandResult { // (Required) Server-side generated resource profile id. int32 profile_id = 1; } + +// Command to remove `CashedRemoteRelation` +message RemoveCachedRemoteRelationCommand { + // (Required) ID of the remote related (assigned by the service). + CachedRemoteRelation relation = 1; +} diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 492dac12f614d..7af99fb3f5bfb 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -39,7 +39,7 @@ import org.apache.spark.connect.proto.Parse.ParseFormat import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance import org.apache.spark.connect.proto.WriteStreamOperationStart.TriggerCase import org.apache.spark.internal.{Logging, MDC} -import org.apache.spark.internal.LogKeys.SESSION_ID +import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.ml.{functions => MLFunctions} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, SparkSession} @@ -2581,6 +2581,8 @@ class SparkConnectPlanner( handleCreateResourceProfileCommand( command.getCreateResourceProfileCommand, responseObserver) + case proto.Command.CommandTypeCase.REMOVE_CACHED_REMOTE_RELATION_COMMAND => + handleRemoveCachedRemoteRelationCommand(command.getRemoveCachedRemoteRelationCommand) case _ => throw new UnsupportedOperationException(s"$command not supported.") } @@ -3507,6 +3509,14 @@ class SparkConnectPlanner( .build()) } + private def handleRemoveCachedRemoteRelationCommand( + removeCachedRemoteRelationCommand: proto.RemoveCachedRemoteRelationCommand): Unit = { + val dfId = removeCachedRemoteRelationCommand.getRelation.getRelationId + logInfo(log"Removing DataFrame with id ${MDC(DATAFRAME_ID, dfId)} from the cache") + sessionHolder.removeCachedDataFrame(dfId) + executeHolder.eventsManager.postFinished() + } + private val emptyLocalRelation = LocalRelation( output = AttributeReference("value", StringType, false)() :: Nil, data = Seq.empty) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 2f35662ea59ac..5bd7225a9d71f 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -106,7 +106,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio // Mapping from relation ID (passed to client) to runtime dataframe. Used for callbacks like // foreachBatch() in Streaming. Lazy since most sessions don't need it. - private lazy val dataFrameCache: ConcurrentMap[String, DataFrame] = new ConcurrentHashMap() + private[spark] lazy val dataFrameCache: ConcurrentMap[String, DataFrame] = new ConcurrentHashMap() // Mapping from id to StreamingQueryListener. Used for methods like removeListener() in // StreamingQueryManager. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index 6c5d95ac67d3d..3df0f8be15f02 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.connect.service +import java.util.UUID + import scala.jdk.CollectionConverters._ import io.grpc.stub.StreamObserver import org.apache.spark.connect.proto -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.LogKeys.DATAFRAME_ID import org.apache.spark.sql.Dataset import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, StorageLevelProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner @@ -206,6 +209,29 @@ private[connect] class SparkConnectAnalyzeHandler( .setStorageLevel(StorageLevelProtoConverter.toConnectProtoType(storageLevel)) .build()) + case proto.AnalyzePlanRequest.AnalyzeCase.CHECKPOINT => + val target = Dataset + .ofRows(session, planner.transformRelation(request.getCheckpoint.getRelation)) + val checkpointed = if (request.getCheckpoint.hasLocal && request.getCheckpoint.hasEager) { + target.localCheckpoint(eager = request.getCheckpoint.getEager) + } else if (request.getCheckpoint.hasLocal) { + target.localCheckpoint() + } else if (request.getCheckpoint.hasEager) { + target.checkpoint(eager = request.getCheckpoint.getEager) + } else { + target.checkpoint() + } + + val dfId = UUID.randomUUID().toString + logInfo(log"Caching DataFrame with id ${MDC(DATAFRAME_ID, dfId)}") + sessionHolder.cacheDataFrameById(dfId, checkpointed) + + builder.setCheckpoint( + proto.AnalyzePlanResponse.Checkpoint + .newBuilder() + .setRelation(proto.CachedRemoteRelation.newBuilder().setRelationId(dfId).build()) + .build()) + case other => throw InvalidPlanInput(s"Unknown Analyze Method $other!") } diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 7b2d6f0bd5fe1..818e6b436b004 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -315,6 +315,12 @@ object SparkConnectService extends Logging { previoslyObservedSessionId) } + // For testing + private[spark] def getOrCreateIsolatedSession( + userId: String, sessionId: String): SessionHolder = { + getOrCreateIsolatedSession(userId, sessionId, None) + } + /** * If there are no executions, return Left with System.currentTimeMillis of last active * execution. Otherwise return Right with list of ExecuteInfo of all executions. diff --git a/python/pyspark/pandas/tests/connect/test_parity_frame_spark.py b/python/pyspark/pandas/tests/connect/test_parity_frame_spark.py index 24626a9164e84..1d285848559bb 100644 --- a/python/pyspark/pandas/tests/connect/test_parity_frame_spark.py +++ b/python/pyspark/pandas/tests/connect/test_parity_frame_spark.py @@ -24,7 +24,7 @@ class SparkFrameMethodsParityTests( SparkFrameMethodsTestsMixin, TestUtils, PandasOnSparkTestUtils, ReusedConnectTestCase ): - @unittest.skip("Test depends on checkpoint which is not supported from Spark Connect.") + @unittest.skip("Test depends on SparkContext which is not supported from Spark Connect.") def test_checkpoint(self): super().test_checkpoint() @@ -32,10 +32,6 @@ def test_checkpoint(self): def test_coalesce(self): super().test_coalesce() - @unittest.skip("Test depends on localCheckpoint which is not supported from Spark Connect.") - def test_local_checkpoint(self): - super().test_local_checkpoint() - @unittest.skip("Test depends on RDD which is not supported from Spark Connect.") def test_repartition(self): super().test_repartition() diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index 5e3462c2d0c1c..23f262c18e487 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -66,7 +66,11 @@ from pyspark.sql.connect.profiler import ConnectProfilerCollector from pyspark.sql.connect.client.reattach import ExecutePlanResponseReattachableIterator from pyspark.sql.connect.client.retries import RetryPolicy, Retrying, DefaultPolicy -from pyspark.sql.connect.conversion import storage_level_to_proto, proto_to_storage_level +from pyspark.sql.connect.conversion import ( + storage_level_to_proto, + proto_to_storage_level, + proto_to_remote_cached_dataframe, +) import pyspark.sql.connect.proto as pb2 import pyspark.sql.connect.proto.base_pb2_grpc as grpc_lib import pyspark.sql.connect.types as types @@ -100,6 +104,7 @@ from google.rpc.error_details_pb2 import ErrorInfo from pyspark.sql.connect._typing import DataTypeOrString from pyspark.sql.datasource import DataSource + from pyspark.sql.connect.dataframe import DataFrame class ChannelBuilder: @@ -528,6 +533,7 @@ def __init__( is_same_semantics: Optional[bool], semantic_hash: Optional[int], storage_level: Optional[StorageLevel], + replaced: Optional["DataFrame"], ): self.schema = schema self.explain_string = explain_string @@ -540,6 +546,7 @@ def __init__( self.is_same_semantics = is_same_semantics self.semantic_hash = semantic_hash self.storage_level = storage_level + self.replaced = replaced @classmethod def fromProto(cls, pb: Any) -> "AnalyzeResult": @@ -554,6 +561,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": is_same_semantics: Optional[bool] = None semantic_hash: Optional[int] = None storage_level: Optional[StorageLevel] = None + replaced: Optional["DataFrame"] = None if pb.HasField("schema"): schema = types.proto_schema_to_pyspark_data_type(pb.schema.schema) @@ -581,6 +589,8 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": pass elif pb.HasField("get_storage_level"): storage_level = proto_to_storage_level(pb.get_storage_level.storage_level) + elif pb.HasField("checkpoint"): + replaced = proto_to_remote_cached_dataframe(pb.checkpoint.relation) else: raise SparkConnectException("No analyze result found!") @@ -596,6 +606,7 @@ def fromProto(cls, pb: Any) -> "AnalyzeResult": is_same_semantics, semantic_hash, storage_level, + replaced, ) @@ -1229,6 +1240,12 @@ def _analyze(self, method: str, **kwargs: Any) -> AnalyzeResult: req.unpersist.blocking = cast(bool, kwargs.get("blocking")) elif method == "get_storage_level": req.get_storage_level.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) + elif method == "checkpoint": + req.checkpoint.relation.CopyFrom(cast(pb2.Relation, kwargs.get("relation"))) + if kwargs.get("local", None) is not None: + req.checkpoint.local = cast(bool, kwargs.get("local")) + if kwargs.get("eager", None) is not None: + req.checkpoint.eager = cast(bool, kwargs.get("eager")) else: raise PySparkValueError( error_class="UNSUPPORTED_OPERATION", diff --git a/python/pyspark/sql/connect/conversion.py b/python/pyspark/sql/connect/conversion.py index 9b1007c41f9c0..b1cf88e40a4e8 100644 --- a/python/pyspark/sql/connect/conversion.py +++ b/python/pyspark/sql/connect/conversion.py @@ -48,12 +48,10 @@ import pyspark.sql.connect.proto as pb2 from pyspark.sql.pandas.types import to_arrow_schema, _dedup_names, _deduplicate_field_names -from typing import ( - Any, - Callable, - Sequence, - List, -) +from typing import Any, Callable, Sequence, List, TYPE_CHECKING + +if TYPE_CHECKING: + from pyspark.sql.connect.dataframe import DataFrame class LocalDataToArrowConversion: @@ -570,3 +568,16 @@ def proto_to_storage_level(storage_level: pb2.StorageLevel) -> StorageLevel: deserialized=storage_level.deserialized, replication=storage_level.replication, ) + + +def proto_to_remote_cached_dataframe(relation: pb2.CachedRemoteRelation) -> "DataFrame": + assert relation is not None and isinstance(relation, pb2.CachedRemoteRelation) + + from pyspark.sql.connect.dataframe import DataFrame + from pyspark.sql.connect.session import SparkSession + import pyspark.sql.connect.plan as plan + + return DataFrame( + plan=plan.CachedRemoteRelation(relation.relation_id), + session=SparkSession.active(), + ) diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index ccaaa15f3190c..5e758cfa57a86 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -16,7 +16,10 @@ # # mypy: disable-error-code="override" +import os +from multiprocessing.pool import ThreadPool +from pyspark.sql.connect.proto import base_pb2 as spark_dot_connect_dot_base__pb2 from pyspark.errors.exceptions.base import ( SessionNotSameException, PySparkIndexError, @@ -104,6 +107,8 @@ class DataFrame(ParentDataFrame): + _release_thread_pool: Optional[ThreadPool] = ThreadPool(os.cpu_count() if os.cpu_count() else 8) + def __new__( cls, plan: plan.LogicalPlan, @@ -137,6 +142,41 @@ def __init__( # by __repr__ and _repr_html_ while eager evaluation opens. self._support_repr_html = False self._cached_schema: Optional[StructType] = None + self._cached_remote_relation_id: Optional[str] = None + + def __del__(self) -> None: + # If session is already closed, all cached DataFrame should be released. + if not self._session.client.is_closed and self._cached_remote_relation_id is not None: + try: + command = plan.RemoveRemoteCachedRelation( + plan.CachedRemoteRelation(relationId=self._cached_remote_relation_id) + ).command(session=self._session.client) + req = self._session.client._execute_plan_request_with_metadata() + if self._session.client._user_id: + req.user_context.user_id = self._session.client._user_id + req.plan.command.CopyFrom(command) + + for attempt in self._session.client._retrying(): + with attempt: + # !!HACK ALERT!! + # unary_stream does not work on Python's exit for an unknown reasons + # Therefore, here we open unary_unary channel instead. + # See also :class:`SparkConnectServiceStub`. + request_serializer = ( + spark_dot_connect_dot_base__pb2.ExecutePlanRequest.SerializeToString + ) + response_deserializer = ( + spark_dot_connect_dot_base__pb2.ExecutePlanResponse.FromString + ) + channel = self._session.client._channel.unary_unary( + "/spark.connect.SparkConnectService/ExecutePlan", + request_serializer=request_serializer, + response_deserializer=response_deserializer, + ) + metadata = self._session.client._builder.metadata() + channel(req, metadata=metadata) # type: ignore[arg-type] + except Exception as e: + warnings.warn(f"RemoveRemoteCachedRelation failed with exception: {e}.") def __reduce__(self) -> Tuple: """ @@ -2095,19 +2135,29 @@ def writeTo(self, table: str) -> "DataFrameWriterV2": def offset(self, n: int) -> ParentDataFrame: return DataFrame(plan.Offset(child=self._plan, offset=n), session=self._session) - if not is_remote_only(): + def checkpoint(self, eager: bool = True) -> "DataFrame": + relation = self._plan.plan(self._session.client) + result = self._session.client._analyze( + method="checkpoint", relation=relation, local=False, eager=eager + ) + assert result.replaced is not None + assert isinstance(result.replaced._plan, plan.CachedRemoteRelation) + checkpointed = result.replaced + checkpointed._cached_remote_relation_id = result.replaced._plan._relationId + return checkpointed - def checkpoint(self, eager: bool = True) -> "DataFrame": - raise PySparkNotImplementedError( - error_class="NOT_IMPLEMENTED", - message_parameters={"feature": "checkpoint()"}, - ) + def localCheckpoint(self, eager: bool = True) -> "DataFrame": + relation = self._plan.plan(self._session.client) + result = self._session.client._analyze( + method="checkpoint", relation=relation, local=False, eager=eager + ) + assert result.replaced is not None + assert isinstance(result.replaced._plan, plan.CachedRemoteRelation) + checkpointed = result.replaced + checkpointed._cached_remote_relation_id = result.replaced._plan._relationId + return checkpointed - def localCheckpoint(self, eager: bool = True) -> "DataFrame": - raise PySparkNotImplementedError( - error_class="NOT_IMPLEMENTED", - message_parameters={"feature": "localCheckpoint()"}, - ) + if not is_remote_only(): def toJSON(self, use_unicode: bool = True) -> "RDD[str]": raise PySparkNotImplementedError( @@ -2202,8 +2252,6 @@ def _test() -> None: if not is_remote_only(): del pyspark.sql.dataframe.DataFrame.toJSON.__doc__ del pyspark.sql.dataframe.DataFrame.rdd.__doc__ - del pyspark.sql.dataframe.DataFrame.checkpoint.__doc__ - del pyspark.sql.dataframe.DataFrame.localCheckpoint.__doc__ globs["spark"] = ( PySparkSession.builder.appName("sql.connect.dataframe tests") diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 3d3303fb15c57..96184ad7165a1 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -1785,9 +1785,20 @@ def command(self, session: "SparkConnectClient") -> proto.Command: return cmd -# Catalog API (internal-only) +class RemoveRemoteCachedRelation(LogicalPlan): + def __init__(self, relation: CachedRemoteRelation) -> None: + super().__init__(None) + self._relation = relation + + def command(self, session: "SparkConnectClient") -> proto.Command: + plan = self._create_proto_relation() + plan.cached_remote_relation.relation_id = self._relation._relationId + cmd = proto.Command() + cmd.remove_cached_remote_relation_command.relation.CopyFrom(plan.cached_remote_relation) + return cmd +# Catalog API (internal-only) class CurrentDatabase(LogicalPlan): def __init__(self) -> None: super().__init__(None) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index a39396db4ff1d..d6bb2e978c1fa 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -37,7 +37,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"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"\xf8\x13\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\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\x08relationB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xce\r\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\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\x0cstorageLevelB\x08\n\x06result"\xa3\x05\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\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\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"\x80\x16\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\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\x1av\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\x42\x0f\n\r_start_offset\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"\x87\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\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\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"\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"\xab\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\x42\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"\x93\x0c\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\x99\x03\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\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_state\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_idx2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"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"\xd6\x15\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\x12N\n\ncheckpoint\x18\x12 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.CheckpointH\x00R\ncheckpoint\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\x8b\x01\n\nCheckpoint\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x19\n\x05local\x18\x02 \x01(\x08H\x00R\x05local\x88\x01\x01\x12\x19\n\x05\x65\x61ger\x18\x03 \x01(\x08H\x01R\x05\x65\x61ger\x88\x01\x01\x42\x08\n\x06_localB\x08\n\x06_eagerB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xee\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\x12O\n\ncheckpoint\x18\x10 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.CheckpointH\x00R\ncheckpoint\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\x1aM\n\nCheckpoint\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relationB\x08\n\x06result"\xa3\x05\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\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\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"\x80\x16\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\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\x1av\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\x42\x0f\n\r_start_offset\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"\x87\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\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\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"\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"\xab\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\x42\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"\x93\x0c\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\x99\x03\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\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_state\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_idx2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -58,171 +58,175 @@ _USERCONTEXT._serialized_start = 337 _USERCONTEXT._serialized_end = 459 _ANALYZEPLANREQUEST._serialized_start = 462 - _ANALYZEPLANREQUEST._serialized_end = 3014 - _ANALYZEPLANREQUEST_SCHEMA._serialized_start = 1745 - _ANALYZEPLANREQUEST_SCHEMA._serialized_end = 1794 - _ANALYZEPLANREQUEST_EXPLAIN._serialized_start = 1797 - _ANALYZEPLANREQUEST_EXPLAIN._serialized_end = 2112 - _ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE._serialized_start = 1940 - _ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE._serialized_end = 2112 - _ANALYZEPLANREQUEST_TREESTRING._serialized_start = 2114 - _ANALYZEPLANREQUEST_TREESTRING._serialized_end = 2204 - _ANALYZEPLANREQUEST_ISLOCAL._serialized_start = 2206 - _ANALYZEPLANREQUEST_ISLOCAL._serialized_end = 2256 - _ANALYZEPLANREQUEST_ISSTREAMING._serialized_start = 2258 - _ANALYZEPLANREQUEST_ISSTREAMING._serialized_end = 2312 - _ANALYZEPLANREQUEST_INPUTFILES._serialized_start = 2314 - _ANALYZEPLANREQUEST_INPUTFILES._serialized_end = 2367 - _ANALYZEPLANREQUEST_SPARKVERSION._serialized_start = 2369 - _ANALYZEPLANREQUEST_SPARKVERSION._serialized_end = 2383 - _ANALYZEPLANREQUEST_DDLPARSE._serialized_start = 2385 - _ANALYZEPLANREQUEST_DDLPARSE._serialized_end = 2426 - _ANALYZEPLANREQUEST_SAMESEMANTICS._serialized_start = 2428 - _ANALYZEPLANREQUEST_SAMESEMANTICS._serialized_end = 2549 - _ANALYZEPLANREQUEST_SEMANTICHASH._serialized_start = 2551 - _ANALYZEPLANREQUEST_SEMANTICHASH._serialized_end = 2606 - _ANALYZEPLANREQUEST_PERSIST._serialized_start = 2609 - _ANALYZEPLANREQUEST_PERSIST._serialized_end = 2760 - _ANALYZEPLANREQUEST_UNPERSIST._serialized_start = 2762 - _ANALYZEPLANREQUEST_UNPERSIST._serialized_end = 2872 - _ANALYZEPLANREQUEST_GETSTORAGELEVEL._serialized_start = 2874 - _ANALYZEPLANREQUEST_GETSTORAGELEVEL._serialized_end = 2944 - _ANALYZEPLANRESPONSE._serialized_start = 3017 - _ANALYZEPLANRESPONSE._serialized_end = 4759 - _ANALYZEPLANRESPONSE_SCHEMA._serialized_start = 4178 - _ANALYZEPLANRESPONSE_SCHEMA._serialized_end = 4235 - _ANALYZEPLANRESPONSE_EXPLAIN._serialized_start = 4237 - _ANALYZEPLANRESPONSE_EXPLAIN._serialized_end = 4285 - _ANALYZEPLANRESPONSE_TREESTRING._serialized_start = 4287 - _ANALYZEPLANRESPONSE_TREESTRING._serialized_end = 4332 - _ANALYZEPLANRESPONSE_ISLOCAL._serialized_start = 4334 - _ANALYZEPLANRESPONSE_ISLOCAL._serialized_end = 4370 - _ANALYZEPLANRESPONSE_ISSTREAMING._serialized_start = 4372 - _ANALYZEPLANRESPONSE_ISSTREAMING._serialized_end = 4420 - _ANALYZEPLANRESPONSE_INPUTFILES._serialized_start = 4422 - _ANALYZEPLANRESPONSE_INPUTFILES._serialized_end = 4456 - _ANALYZEPLANRESPONSE_SPARKVERSION._serialized_start = 4458 - _ANALYZEPLANRESPONSE_SPARKVERSION._serialized_end = 4498 - _ANALYZEPLANRESPONSE_DDLPARSE._serialized_start = 4500 - _ANALYZEPLANRESPONSE_DDLPARSE._serialized_end = 4559 - _ANALYZEPLANRESPONSE_SAMESEMANTICS._serialized_start = 4561 - _ANALYZEPLANRESPONSE_SAMESEMANTICS._serialized_end = 4600 - _ANALYZEPLANRESPONSE_SEMANTICHASH._serialized_start = 4602 - _ANALYZEPLANRESPONSE_SEMANTICHASH._serialized_end = 4640 - _ANALYZEPLANRESPONSE_PERSIST._serialized_start = 2609 - _ANALYZEPLANRESPONSE_PERSIST._serialized_end = 2618 - _ANALYZEPLANRESPONSE_UNPERSIST._serialized_start = 2762 - _ANALYZEPLANRESPONSE_UNPERSIST._serialized_end = 2773 - _ANALYZEPLANRESPONSE_GETSTORAGELEVEL._serialized_start = 4666 - _ANALYZEPLANRESPONSE_GETSTORAGELEVEL._serialized_end = 4749 - _EXECUTEPLANREQUEST._serialized_start = 4762 - _EXECUTEPLANREQUEST._serialized_end = 5437 - _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_start = 5196 - _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_end = 5361 - _EXECUTEPLANRESPONSE._serialized_start = 5440 - _EXECUTEPLANRESPONSE._serialized_end = 8256 - _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 7030 - _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 7101 - _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 7103 - _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 7221 - _EXECUTEPLANRESPONSE_METRICS._serialized_start = 7224 - _EXECUTEPLANRESPONSE_METRICS._serialized_end = 7741 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 7319 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 7651 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 7528 - _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 7651 - _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 7653 - _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 7741 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 7744 - _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 7885 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 7887 - _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 7903 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 7906 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8239 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8062 - _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8239 - _KEYVALUE._serialized_start = 8258 - _KEYVALUE._serialized_end = 8323 - _CONFIGREQUEST._serialized_start = 8326 - _CONFIGREQUEST._serialized_end = 9485 - _CONFIGREQUEST_OPERATION._serialized_start = 8634 - _CONFIGREQUEST_OPERATION._serialized_end = 9132 - _CONFIGREQUEST_SET._serialized_start = 9134 - _CONFIGREQUEST_SET._serialized_end = 9186 - _CONFIGREQUEST_GET._serialized_start = 9188 - _CONFIGREQUEST_GET._serialized_end = 9213 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9215 - _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9278 - _CONFIGREQUEST_GETOPTION._serialized_start = 9280 - _CONFIGREQUEST_GETOPTION._serialized_end = 9311 - _CONFIGREQUEST_GETALL._serialized_start = 9313 - _CONFIGREQUEST_GETALL._serialized_end = 9361 - _CONFIGREQUEST_UNSET._serialized_start = 9363 - _CONFIGREQUEST_UNSET._serialized_end = 9390 - _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9392 - _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9426 - _CONFIGRESPONSE._serialized_start = 9488 - _CONFIGRESPONSE._serialized_end = 9663 - _ADDARTIFACTSREQUEST._serialized_start = 9666 - _ADDARTIFACTSREQUEST._serialized_end = 10668 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10141 - _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10194 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10196 - _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10307 - _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10309 - _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10402 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10405 - _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10598 - _ADDARTIFACTSRESPONSE._serialized_start = 10671 - _ADDARTIFACTSRESPONSE._serialized_end = 10943 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 10862 - _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 10943 - _ARTIFACTSTATUSESREQUEST._serialized_start = 10946 - _ARTIFACTSTATUSESREQUEST._serialized_end = 11272 - _ARTIFACTSTATUSESRESPONSE._serialized_start = 11275 - _ARTIFACTSTATUSESRESPONSE._serialized_end = 11627 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11470 - _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11585 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11587 - _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 11627 - _INTERRUPTREQUEST._serialized_start = 11630 - _INTERRUPTREQUEST._serialized_end = 12233 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12033 - _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12161 - _INTERRUPTRESPONSE._serialized_start = 12236 - _INTERRUPTRESPONSE._serialized_end = 12380 - _REATTACHOPTIONS._serialized_start = 12382 - _REATTACHOPTIONS._serialized_end = 12435 - _REATTACHEXECUTEREQUEST._serialized_start = 12438 - _REATTACHEXECUTEREQUEST._serialized_end = 12844 - _RELEASEEXECUTEREQUEST._serialized_start = 12847 - _RELEASEEXECUTEREQUEST._serialized_end = 13432 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13301 - _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13313 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13315 - _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13362 - _RELEASEEXECUTERESPONSE._serialized_start = 13435 - _RELEASEEXECUTERESPONSE._serialized_end = 13600 - _RELEASESESSIONREQUEST._serialized_start = 13603 - _RELEASESESSIONREQUEST._serialized_end = 13774 - _RELEASESESSIONRESPONSE._serialized_start = 13776 - _RELEASESESSIONRESPONSE._serialized_end = 13884 - _FETCHERRORDETAILSREQUEST._serialized_start = 13887 - _FETCHERRORDETAILSREQUEST._serialized_end = 14219 - _FETCHERRORDETAILSRESPONSE._serialized_start = 14222 - _FETCHERRORDETAILSRESPONSE._serialized_end = 15777 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14451 - _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 14625 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 14628 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 14996 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 14959 - _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 14996 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 14999 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15408 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15310 - _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15378 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15411 - _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 15758 - _SPARKCONNECTSERVICE._serialized_start = 15780 - _SPARKCONNECTSERVICE._serialized_end = 16726 + _ANALYZEPLANREQUEST._serialized_end = 3236 + _ANALYZEPLANREQUEST_SCHEMA._serialized_start = 1825 + _ANALYZEPLANREQUEST_SCHEMA._serialized_end = 1874 + _ANALYZEPLANREQUEST_EXPLAIN._serialized_start = 1877 + _ANALYZEPLANREQUEST_EXPLAIN._serialized_end = 2192 + _ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE._serialized_start = 2020 + _ANALYZEPLANREQUEST_EXPLAIN_EXPLAINMODE._serialized_end = 2192 + _ANALYZEPLANREQUEST_TREESTRING._serialized_start = 2194 + _ANALYZEPLANREQUEST_TREESTRING._serialized_end = 2284 + _ANALYZEPLANREQUEST_ISLOCAL._serialized_start = 2286 + _ANALYZEPLANREQUEST_ISLOCAL._serialized_end = 2336 + _ANALYZEPLANREQUEST_ISSTREAMING._serialized_start = 2338 + _ANALYZEPLANREQUEST_ISSTREAMING._serialized_end = 2392 + _ANALYZEPLANREQUEST_INPUTFILES._serialized_start = 2394 + _ANALYZEPLANREQUEST_INPUTFILES._serialized_end = 2447 + _ANALYZEPLANREQUEST_SPARKVERSION._serialized_start = 2449 + _ANALYZEPLANREQUEST_SPARKVERSION._serialized_end = 2463 + _ANALYZEPLANREQUEST_DDLPARSE._serialized_start = 2465 + _ANALYZEPLANREQUEST_DDLPARSE._serialized_end = 2506 + _ANALYZEPLANREQUEST_SAMESEMANTICS._serialized_start = 2508 + _ANALYZEPLANREQUEST_SAMESEMANTICS._serialized_end = 2629 + _ANALYZEPLANREQUEST_SEMANTICHASH._serialized_start = 2631 + _ANALYZEPLANREQUEST_SEMANTICHASH._serialized_end = 2686 + _ANALYZEPLANREQUEST_PERSIST._serialized_start = 2689 + _ANALYZEPLANREQUEST_PERSIST._serialized_end = 2840 + _ANALYZEPLANREQUEST_UNPERSIST._serialized_start = 2842 + _ANALYZEPLANREQUEST_UNPERSIST._serialized_end = 2952 + _ANALYZEPLANREQUEST_GETSTORAGELEVEL._serialized_start = 2954 + _ANALYZEPLANREQUEST_GETSTORAGELEVEL._serialized_end = 3024 + _ANALYZEPLANREQUEST_CHECKPOINT._serialized_start = 3027 + _ANALYZEPLANREQUEST_CHECKPOINT._serialized_end = 3166 + _ANALYZEPLANRESPONSE._serialized_start = 3239 + _ANALYZEPLANRESPONSE._serialized_end = 5141 + _ANALYZEPLANRESPONSE_SCHEMA._serialized_start = 4481 + _ANALYZEPLANRESPONSE_SCHEMA._serialized_end = 4538 + _ANALYZEPLANRESPONSE_EXPLAIN._serialized_start = 4540 + _ANALYZEPLANRESPONSE_EXPLAIN._serialized_end = 4588 + _ANALYZEPLANRESPONSE_TREESTRING._serialized_start = 4590 + _ANALYZEPLANRESPONSE_TREESTRING._serialized_end = 4635 + _ANALYZEPLANRESPONSE_ISLOCAL._serialized_start = 4637 + _ANALYZEPLANRESPONSE_ISLOCAL._serialized_end = 4673 + _ANALYZEPLANRESPONSE_ISSTREAMING._serialized_start = 4675 + _ANALYZEPLANRESPONSE_ISSTREAMING._serialized_end = 4723 + _ANALYZEPLANRESPONSE_INPUTFILES._serialized_start = 4725 + _ANALYZEPLANRESPONSE_INPUTFILES._serialized_end = 4759 + _ANALYZEPLANRESPONSE_SPARKVERSION._serialized_start = 4761 + _ANALYZEPLANRESPONSE_SPARKVERSION._serialized_end = 4801 + _ANALYZEPLANRESPONSE_DDLPARSE._serialized_start = 4803 + _ANALYZEPLANRESPONSE_DDLPARSE._serialized_end = 4862 + _ANALYZEPLANRESPONSE_SAMESEMANTICS._serialized_start = 4864 + _ANALYZEPLANRESPONSE_SAMESEMANTICS._serialized_end = 4903 + _ANALYZEPLANRESPONSE_SEMANTICHASH._serialized_start = 4905 + _ANALYZEPLANRESPONSE_SEMANTICHASH._serialized_end = 4943 + _ANALYZEPLANRESPONSE_PERSIST._serialized_start = 2689 + _ANALYZEPLANRESPONSE_PERSIST._serialized_end = 2698 + _ANALYZEPLANRESPONSE_UNPERSIST._serialized_start = 2842 + _ANALYZEPLANRESPONSE_UNPERSIST._serialized_end = 2853 + _ANALYZEPLANRESPONSE_GETSTORAGELEVEL._serialized_start = 4969 + _ANALYZEPLANRESPONSE_GETSTORAGELEVEL._serialized_end = 5052 + _ANALYZEPLANRESPONSE_CHECKPOINT._serialized_start = 5054 + _ANALYZEPLANRESPONSE_CHECKPOINT._serialized_end = 5131 + _EXECUTEPLANREQUEST._serialized_start = 5144 + _EXECUTEPLANREQUEST._serialized_end = 5819 + _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_start = 5578 + _EXECUTEPLANREQUEST_REQUESTOPTION._serialized_end = 5743 + _EXECUTEPLANRESPONSE._serialized_start = 5822 + _EXECUTEPLANRESPONSE._serialized_end = 8638 + _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_start = 7412 + _EXECUTEPLANRESPONSE_SQLCOMMANDRESULT._serialized_end = 7483 + _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_start = 7485 + _EXECUTEPLANRESPONSE_ARROWBATCH._serialized_end = 7603 + _EXECUTEPLANRESPONSE_METRICS._serialized_start = 7606 + _EXECUTEPLANRESPONSE_METRICS._serialized_end = 8123 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_start = 7701 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT._serialized_end = 8033 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_start = 7910 + _EXECUTEPLANRESPONSE_METRICS_METRICOBJECT_EXECUTIONMETRICSENTRY._serialized_end = 8033 + _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_start = 8035 + _EXECUTEPLANRESPONSE_METRICS_METRICVALUE._serialized_end = 8123 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_start = 8126 + _EXECUTEPLANRESPONSE_OBSERVEDMETRICS._serialized_end = 8267 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_start = 8269 + _EXECUTEPLANRESPONSE_RESULTCOMPLETE._serialized_end = 8285 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_start = 8288 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS._serialized_end = 8621 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_start = 8444 + _EXECUTEPLANRESPONSE_EXECUTIONPROGRESS_STAGEINFO._serialized_end = 8621 + _KEYVALUE._serialized_start = 8640 + _KEYVALUE._serialized_end = 8705 + _CONFIGREQUEST._serialized_start = 8708 + _CONFIGREQUEST._serialized_end = 9867 + _CONFIGREQUEST_OPERATION._serialized_start = 9016 + _CONFIGREQUEST_OPERATION._serialized_end = 9514 + _CONFIGREQUEST_SET._serialized_start = 9516 + _CONFIGREQUEST_SET._serialized_end = 9568 + _CONFIGREQUEST_GET._serialized_start = 9570 + _CONFIGREQUEST_GET._serialized_end = 9595 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_start = 9597 + _CONFIGREQUEST_GETWITHDEFAULT._serialized_end = 9660 + _CONFIGREQUEST_GETOPTION._serialized_start = 9662 + _CONFIGREQUEST_GETOPTION._serialized_end = 9693 + _CONFIGREQUEST_GETALL._serialized_start = 9695 + _CONFIGREQUEST_GETALL._serialized_end = 9743 + _CONFIGREQUEST_UNSET._serialized_start = 9745 + _CONFIGREQUEST_UNSET._serialized_end = 9772 + _CONFIGREQUEST_ISMODIFIABLE._serialized_start = 9774 + _CONFIGREQUEST_ISMODIFIABLE._serialized_end = 9808 + _CONFIGRESPONSE._serialized_start = 9870 + _CONFIGRESPONSE._serialized_end = 10045 + _ADDARTIFACTSREQUEST._serialized_start = 10048 + _ADDARTIFACTSREQUEST._serialized_end = 11050 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_start = 10523 + _ADDARTIFACTSREQUEST_ARTIFACTCHUNK._serialized_end = 10576 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_start = 10578 + _ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT._serialized_end = 10689 + _ADDARTIFACTSREQUEST_BATCH._serialized_start = 10691 + _ADDARTIFACTSREQUEST_BATCH._serialized_end = 10784 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_start = 10787 + _ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT._serialized_end = 10980 + _ADDARTIFACTSRESPONSE._serialized_start = 11053 + _ADDARTIFACTSRESPONSE._serialized_end = 11325 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_start = 11244 + _ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY._serialized_end = 11325 + _ARTIFACTSTATUSESREQUEST._serialized_start = 11328 + _ARTIFACTSTATUSESREQUEST._serialized_end = 11654 + _ARTIFACTSTATUSESRESPONSE._serialized_start = 11657 + _ARTIFACTSTATUSESRESPONSE._serialized_end = 12009 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_start = 11852 + _ARTIFACTSTATUSESRESPONSE_STATUSESENTRY._serialized_end = 11967 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_start = 11969 + _ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS._serialized_end = 12009 + _INTERRUPTREQUEST._serialized_start = 12012 + _INTERRUPTREQUEST._serialized_end = 12615 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_start = 12415 + _INTERRUPTREQUEST_INTERRUPTTYPE._serialized_end = 12543 + _INTERRUPTRESPONSE._serialized_start = 12618 + _INTERRUPTRESPONSE._serialized_end = 12762 + _REATTACHOPTIONS._serialized_start = 12764 + _REATTACHOPTIONS._serialized_end = 12817 + _REATTACHEXECUTEREQUEST._serialized_start = 12820 + _REATTACHEXECUTEREQUEST._serialized_end = 13226 + _RELEASEEXECUTEREQUEST._serialized_start = 13229 + _RELEASEEXECUTEREQUEST._serialized_end = 13814 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_start = 13683 + _RELEASEEXECUTEREQUEST_RELEASEALL._serialized_end = 13695 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_start = 13697 + _RELEASEEXECUTEREQUEST_RELEASEUNTIL._serialized_end = 13744 + _RELEASEEXECUTERESPONSE._serialized_start = 13817 + _RELEASEEXECUTERESPONSE._serialized_end = 13982 + _RELEASESESSIONREQUEST._serialized_start = 13985 + _RELEASESESSIONREQUEST._serialized_end = 14156 + _RELEASESESSIONRESPONSE._serialized_start = 14158 + _RELEASESESSIONRESPONSE._serialized_end = 14266 + _FETCHERRORDETAILSREQUEST._serialized_start = 14269 + _FETCHERRORDETAILSREQUEST._serialized_end = 14601 + _FETCHERRORDETAILSRESPONSE._serialized_start = 14604 + _FETCHERRORDETAILSRESPONSE._serialized_end = 16159 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_start = 14833 + _FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT._serialized_end = 15007 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_start = 15010 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT._serialized_end = 15378 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_start = 15341 + _FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE._serialized_end = 15378 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_start = 15381 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE._serialized_end = 15790 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_start = 15692 + _FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY._serialized_end = 15760 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_start = 15793 + _FETCHERRORDETAILSRESPONSE_ERROR._serialized_end = 16140 + _SPARKCONNECTSERVICE._serialized_start = 16162 + _SPARKCONNECTSERVICE._serialized_end = 17108 # @@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 b76f2a7f4de34..95c239879d37d 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -477,6 +477,65 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... + class Checkpoint(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + RELATION_FIELD_NUMBER: builtins.int + LOCAL_FIELD_NUMBER: builtins.int + EAGER_FIELD_NUMBER: builtins.int + @property + def relation(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: + """(Required) The logical plan to checkpoint.""" + local: builtins.bool + """(Optional) Is it localCheckpoint""" + eager: builtins.bool + """(Optional) Is it eager""" + def __init__( + self, + *, + relation: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., + local: builtins.bool | None = ..., + eager: builtins.bool | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_eager", + b"_eager", + "_local", + b"_local", + "eager", + b"eager", + "local", + b"local", + "relation", + b"relation", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_eager", + b"_eager", + "_local", + b"_local", + "eager", + b"eager", + "local", + b"local", + "relation", + b"relation", + ], + ) -> None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_eager", b"_eager"] + ) -> typing_extensions.Literal["eager"] | None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_local", b"_local"] + ) -> typing_extensions.Literal["local"] | None: ... + SESSION_ID_FIELD_NUMBER: builtins.int CLIENT_OBSERVED_SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int @@ -494,6 +553,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): PERSIST_FIELD_NUMBER: builtins.int UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int + CHECKPOINT_FIELD_NUMBER: builtins.int session_id: builtins.str """(Required) @@ -542,6 +602,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): def unpersist(self) -> global___AnalyzePlanRequest.Unpersist: ... @property def get_storage_level(self) -> global___AnalyzePlanRequest.GetStorageLevel: ... + @property + def checkpoint(self) -> global___AnalyzePlanRequest.Checkpoint: ... def __init__( self, *, @@ -562,6 +624,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): persist: global___AnalyzePlanRequest.Persist | None = ..., unpersist: global___AnalyzePlanRequest.Unpersist | None = ..., get_storage_level: global___AnalyzePlanRequest.GetStorageLevel | None = ..., + checkpoint: global___AnalyzePlanRequest.Checkpoint | None = ..., ) -> None: ... def HasField( self, @@ -572,6 +635,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"_client_type", "analyze", b"analyze", + "checkpoint", + b"checkpoint", "client_observed_server_side_session_id", b"client_observed_server_side_session_id", "client_type", @@ -615,6 +680,8 @@ class AnalyzePlanRequest(google.protobuf.message.Message): b"_client_type", "analyze", b"analyze", + "checkpoint", + b"checkpoint", "client_observed_server_side_session_id", b"client_observed_server_side_session_id", "client_type", @@ -680,6 +747,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): "persist", "unpersist", "get_storage_level", + "checkpoint", ] | None ): ... @@ -877,6 +945,25 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["storage_level", b"storage_level"] ) -> None: ... + class Checkpoint(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + RELATION_FIELD_NUMBER: builtins.int + @property + def relation(self) -> pyspark.sql.connect.proto.relations_pb2.CachedRemoteRelation: + """(Required) The logical plan checkpointed.""" + def __init__( + self, + *, + relation: pyspark.sql.connect.proto.relations_pb2.CachedRemoteRelation | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["relation", b"relation"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["relation", b"relation"] + ) -> None: ... + SESSION_ID_FIELD_NUMBER: builtins.int SERVER_SIDE_SESSION_ID_FIELD_NUMBER: builtins.int SCHEMA_FIELD_NUMBER: builtins.int @@ -892,6 +979,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): PERSIST_FIELD_NUMBER: builtins.int UNPERSIST_FIELD_NUMBER: builtins.int GET_STORAGE_LEVEL_FIELD_NUMBER: builtins.int + CHECKPOINT_FIELD_NUMBER: builtins.int session_id: builtins.str server_side_session_id: builtins.str """Server-side generated idempotency key that the client can use to assert that the server side @@ -923,6 +1011,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): def unpersist(self) -> global___AnalyzePlanResponse.Unpersist: ... @property def get_storage_level(self) -> global___AnalyzePlanResponse.GetStorageLevel: ... + @property + def checkpoint(self) -> global___AnalyzePlanResponse.Checkpoint: ... def __init__( self, *, @@ -941,10 +1031,13 @@ class AnalyzePlanResponse(google.protobuf.message.Message): persist: global___AnalyzePlanResponse.Persist | None = ..., unpersist: global___AnalyzePlanResponse.Unpersist | None = ..., get_storage_level: global___AnalyzePlanResponse.GetStorageLevel | None = ..., + checkpoint: global___AnalyzePlanResponse.Checkpoint | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ + "checkpoint", + b"checkpoint", "ddl_parse", b"ddl_parse", "explain", @@ -978,6 +1071,8 @@ class AnalyzePlanResponse(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ + "checkpoint", + b"checkpoint", "ddl_parse", b"ddl_parse", "explain", @@ -1029,6 +1124,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): "persist", "unpersist", "get_storage_level", + "checkpoint", ] | None ): ... diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 50cd309dcd8da..f6625443b10a3 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xd5\n\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xdc\x0b\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0e \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -71,98 +71,100 @@ _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_options = b"8\001" _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._options = None _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_options = b"8\001" - _STREAMINGQUERYEVENTTYPE._serialized_start = 10080 - _STREAMINGQUERYEVENTTYPE._serialized_end = 10213 + _STREAMINGQUERYEVENTTYPE._serialized_start = 10317 + _STREAMINGQUERYEVENTTYPE._serialized_end = 10450 _COMMAND._serialized_start = 167 - _COMMAND._serialized_end = 1532 - _SQLCOMMAND._serialized_start = 1535 - _SQLCOMMAND._serialized_end = 2089 - _SQLCOMMAND_ARGSENTRY._serialized_start = 1905 - _SQLCOMMAND_ARGSENTRY._serialized_end = 1995 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 1997 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 2089 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 2092 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 2242 - _WRITEOPERATION._serialized_start = 2245 - _WRITEOPERATION._serialized_end = 3343 - _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2767 - _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2825 - _WRITEOPERATION_SAVETABLE._serialized_start = 2828 - _WRITEOPERATION_SAVETABLE._serialized_end = 3086 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 2962 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 3086 - _WRITEOPERATION_BUCKETBY._serialized_start = 3088 - _WRITEOPERATION_BUCKETBY._serialized_end = 3179 - _WRITEOPERATION_SAVEMODE._serialized_start = 3182 - _WRITEOPERATION_SAVEMODE._serialized_end = 3319 - _WRITEOPERATIONV2._serialized_start = 3346 - _WRITEOPERATIONV2._serialized_end = 4206 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2767 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2825 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 3965 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 4031 - _WRITEOPERATIONV2_MODE._serialized_start = 4034 - _WRITEOPERATIONV2_MODE._serialized_end = 4193 - _WRITESTREAMOPERATIONSTART._serialized_start = 4209 - _WRITESTREAMOPERATIONSTART._serialized_end = 5009 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2767 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2825 - _STREAMINGFOREACHFUNCTION._serialized_start = 5012 - _STREAMINGFOREACHFUNCTION._serialized_end = 5191 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 5194 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 5406 - _STREAMINGQUERYINSTANCEID._serialized_start = 5408 - _STREAMINGQUERYINSTANCEID._serialized_end = 5473 - _STREAMINGQUERYCOMMAND._serialized_start = 5476 - _STREAMINGQUERYCOMMAND._serialized_end = 6108 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 5975 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 6019 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 6021 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 6097 - _STREAMINGQUERYCOMMANDRESULT._serialized_start = 6111 - _STREAMINGQUERYCOMMANDRESULT._serialized_end = 7252 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6694 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6864 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 6866 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 6938 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 6940 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 6979 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 6982 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 7179 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 7181 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 7237 - _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 7255 - _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 8084 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7786 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 7865 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 7868 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 8073 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 8087 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 9163 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8695 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8822 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8824 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 8939 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 8941 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 9000 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 9002 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 9077 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 9079 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 9148 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 9166 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 9339 - _STREAMINGQUERYLISTENEREVENT._serialized_start = 9342 - _STREAMINGQUERYLISTENEREVENT._serialized_end = 9473 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9476 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9680 - _GETRESOURCESCOMMAND._serialized_start = 9682 - _GETRESOURCESCOMMAND._serialized_end = 9703 - _GETRESOURCESCOMMANDRESULT._serialized_start = 9706 - _GETRESOURCESCOMMANDRESULT._serialized_end = 9918 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 9822 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 9918 - _CREATERESOURCEPROFILECOMMAND._serialized_start = 9920 - _CREATERESOURCEPROFILECOMMAND._serialized_end = 10008 - _CREATERESOURCEPROFILECOMMANDRESULT._serialized_start = 10010 - _CREATERESOURCEPROFILECOMMANDRESULT._serialized_end = 10077 + _COMMAND._serialized_end = 1667 + _SQLCOMMAND._serialized_start = 1670 + _SQLCOMMAND._serialized_end = 2224 + _SQLCOMMAND_ARGSENTRY._serialized_start = 2040 + _SQLCOMMAND_ARGSENTRY._serialized_end = 2130 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 2132 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 2224 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 2227 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 2377 + _WRITEOPERATION._serialized_start = 2380 + _WRITEOPERATION._serialized_end = 3478 + _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2902 + _WRITEOPERATION_OPTIONSENTRY._serialized_end = 2960 + _WRITEOPERATION_SAVETABLE._serialized_start = 2963 + _WRITEOPERATION_SAVETABLE._serialized_end = 3221 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 3097 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 3221 + _WRITEOPERATION_BUCKETBY._serialized_start = 3223 + _WRITEOPERATION_BUCKETBY._serialized_end = 3314 + _WRITEOPERATION_SAVEMODE._serialized_start = 3317 + _WRITEOPERATION_SAVEMODE._serialized_end = 3454 + _WRITEOPERATIONV2._serialized_start = 3481 + _WRITEOPERATIONV2._serialized_end = 4341 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2902 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 2960 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 4100 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 4166 + _WRITEOPERATIONV2_MODE._serialized_start = 4169 + _WRITEOPERATIONV2_MODE._serialized_end = 4328 + _WRITESTREAMOPERATIONSTART._serialized_start = 4344 + _WRITESTREAMOPERATIONSTART._serialized_end = 5144 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2902 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 2960 + _STREAMINGFOREACHFUNCTION._serialized_start = 5147 + _STREAMINGFOREACHFUNCTION._serialized_end = 5326 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 5329 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 5541 + _STREAMINGQUERYINSTANCEID._serialized_start = 5543 + _STREAMINGQUERYINSTANCEID._serialized_end = 5608 + _STREAMINGQUERYCOMMAND._serialized_start = 5611 + _STREAMINGQUERYCOMMAND._serialized_end = 6243 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 6110 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 6154 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 6156 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 6232 + _STREAMINGQUERYCOMMANDRESULT._serialized_start = 6246 + _STREAMINGQUERYCOMMANDRESULT._serialized_end = 7387 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6829 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 6999 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 7001 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 7073 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 7075 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 7114 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 7117 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 7314 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 7316 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 7372 + _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 7390 + _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 8219 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 7921 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 8000 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 8003 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 8208 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 8222 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 9298 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8830 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 8957 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 8959 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 9074 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 9076 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 9135 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 9137 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 9212 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 9214 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 9283 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 9301 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 9474 + _STREAMINGQUERYLISTENEREVENT._serialized_start = 9477 + _STREAMINGQUERYLISTENEREVENT._serialized_end = 9608 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9611 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9815 + _GETRESOURCESCOMMAND._serialized_start = 9817 + _GETRESOURCESCOMMAND._serialized_end = 9838 + _GETRESOURCESCOMMANDRESULT._serialized_start = 9841 + _GETRESOURCESCOMMANDRESULT._serialized_end = 10053 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 9957 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 10053 + _CREATERESOURCEPROFILECOMMAND._serialized_start = 10055 + _CREATERESOURCEPROFILECOMMAND._serialized_end = 10143 + _CREATERESOURCEPROFILECOMMANDRESULT._serialized_start = 10145 + _CREATERESOURCEPROFILECOMMANDRESULT._serialized_end = 10212 + _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_start = 10214 + _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_end = 10314 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index f86ae653508e3..870015a036b28 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -101,6 +101,7 @@ class Command(google.protobuf.message.Message): STREAMING_QUERY_LISTENER_BUS_COMMAND_FIELD_NUMBER: builtins.int REGISTER_DATA_SOURCE_FIELD_NUMBER: builtins.int CREATE_RESOURCE_PROFILE_COMMAND_FIELD_NUMBER: builtins.int + REMOVE_CACHED_REMOTE_RELATION_COMMAND_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def register_function( @@ -135,6 +136,10 @@ class Command(google.protobuf.message.Message): @property def create_resource_profile_command(self) -> global___CreateResourceProfileCommand: ... @property + def remove_cached_remote_relation_command( + self, + ) -> global___RemoveCachedRemoteRelationCommand: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary Commands they can add them here. During the planning the correct resolution is done. @@ -159,6 +164,8 @@ class Command(google.protobuf.message.Message): register_data_source: pyspark.sql.connect.proto.relations_pb2.CommonInlineUserDefinedDataSource | None = ..., create_resource_profile_command: global___CreateResourceProfileCommand | None = ..., + remove_cached_remote_relation_command: global___RemoveCachedRemoteRelationCommand + | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -180,6 +187,8 @@ class Command(google.protobuf.message.Message): b"register_function", "register_table_function", b"register_table_function", + "remove_cached_remote_relation_command", + b"remove_cached_remote_relation_command", "sql_command", b"sql_command", "streaming_query_command", @@ -215,6 +224,8 @@ class Command(google.protobuf.message.Message): b"register_function", "register_table_function", b"register_table_function", + "remove_cached_remote_relation_command", + b"remove_cached_remote_relation_command", "sql_command", b"sql_command", "streaming_query_command", @@ -248,6 +259,7 @@ class Command(google.protobuf.message.Message): "streaming_query_listener_bus_command", "register_data_source", "create_resource_profile_command", + "remove_cached_remote_relation_command", "extension", ] | None @@ -2119,3 +2131,26 @@ class CreateResourceProfileCommandResult(google.protobuf.message.Message): ) -> None: ... global___CreateResourceProfileCommandResult = CreateResourceProfileCommandResult + +class RemoveCachedRemoteRelationCommand(google.protobuf.message.Message): + """Command to remove `CashedRemoteRelation`""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + RELATION_FIELD_NUMBER: builtins.int + @property + def relation(self) -> pyspark.sql.connect.proto.relations_pb2.CachedRemoteRelation: + """(Required) ID of the remote related (assigned by the service).""" + def __init__( + self, + *, + relation: pyspark.sql.connect.proto.relations_pb2.CachedRemoteRelation | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["relation", b"relation"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["relation", b"relation"] + ) -> None: ... + +global___RemoveCachedRemoteRelationCommand = RemoveCachedRemoteRelationCommand diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 886f72cc371e9..51d8188066c6f 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -976,76 +976,78 @@ def _repr_html_(self) -> Optional[str]: """ ... - if not is_remote_only(): + def checkpoint(self, eager: bool = True) -> "DataFrame": + """Returns a checkpointed version of this :class:`DataFrame`. Checkpointing can be + used to truncate the logical plan of this :class:`DataFrame`, which is especially + useful in iterative algorithms where the plan may grow exponentially. It will be + saved to files inside the checkpoint directory set with + :meth:`SparkContext.setCheckpointDir`. - def checkpoint(self, eager: bool = True) -> "DataFrame": - """Returns a checkpointed version of this :class:`DataFrame`. Checkpointing can be - used to truncate the logical plan of this :class:`DataFrame`, which is especially - useful in iterative algorithms where the plan may grow exponentially. It will be - saved to files inside the checkpoint directory set with - :meth:`SparkContext.setCheckpointDir`. + .. versionadded:: 2.1.0 - .. versionadded:: 2.1.0 + .. versionchanged:: 4.0.0 + Supports Spark Connect. - Parameters - ---------- - eager : bool, optional, default True - Whether to checkpoint this :class:`DataFrame` immediately. + Parameters + ---------- + eager : bool, optional, default True + Whether to checkpoint this :class:`DataFrame` immediately. - Returns - ------- - :class:`DataFrame` - Checkpointed DataFrame. + Returns + ------- + :class:`DataFrame` + Checkpointed DataFrame. - Notes - ----- - This API is experimental. + Notes + ----- + This API is experimental. - Examples - -------- - >>> import tempfile - >>> df = spark.createDataFrame([ - ... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"]) - >>> with tempfile.TemporaryDirectory(prefix="checkpoint") as d: - ... spark.sparkContext.setCheckpointDir("/tmp/bb") - ... df.checkpoint(False) - DataFrame[age: bigint, name: string] - """ - ... + Examples + -------- + >>> import tempfile + >>> df = spark.createDataFrame([ + ... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"]) + >>> with tempfile.TemporaryDirectory(prefix="checkpoint") as d: + ... spark.sparkContext.setCheckpointDir("/tmp/bb") + ... df.checkpoint(False) + DataFrame[age: bigint, name: string] + """ + ... - if not is_remote_only(): + def localCheckpoint(self, eager: bool = True) -> "DataFrame": + """Returns a locally checkpointed version of this :class:`DataFrame`. Checkpointing can + be used to truncate the logical plan of this :class:`DataFrame`, which is especially + useful in iterative algorithms where the plan may grow exponentially. Local checkpoints + are stored in the executors using the caching subsystem and therefore they are not + reliable. - def localCheckpoint(self, eager: bool = True) -> "DataFrame": - """Returns a locally checkpointed version of this :class:`DataFrame`. Checkpointing can - be used to truncate the logical plan of this :class:`DataFrame`, which is especially - useful in iterative algorithms where the plan may grow exponentially. Local checkpoints - are stored in the executors using the caching subsystem and therefore they are not - reliable. + .. versionadded:: 2.3.0 - .. versionadded:: 2.3.0 + .. versionchanged:: 4.0.0 + Supports Spark Connect. - Parameters - ---------- - eager : bool, optional, default True - Whether to checkpoint this :class:`DataFrame` immediately. + Parameters + ---------- + eager : bool, optional, default True + Whether to checkpoint this :class:`DataFrame` immediately. - Returns - ------- - :class:`DataFrame` - Checkpointed DataFrame. + Returns + ------- + :class:`DataFrame` + Checkpointed DataFrame. - Notes - ----- - This API is experimental. + Notes + ----- + This API is experimental. - Examples - -------- - >>> df = spark.createDataFrame([ - ... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"]) - >>> df.localCheckpoint(False) - DataFrame[age: bigint, name: string] - """ - ... + Examples + -------- + >>> df = spark.createDataFrame([ + ... (14, "Tom"), (23, "Alice"), (16, "Bob")], ["age", "name"]) + >>> df.localCheckpoint(False) + DataFrame[age: bigint, name: string] + """ + ... @dispatch_df_method def withWatermark(self, eventTime: str, delayThreshold: str) -> "DataFrame": diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 0d84764f53602..93bd4caf57d42 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -19,6 +19,7 @@ import unittest import shutil import tempfile +import time from pyspark.util import is_remote_only from pyspark.errors import PySparkTypeError, PySparkValueError @@ -1358,6 +1359,37 @@ def test_verify_col_name(self): self.assertTrue(verify_col_name("`m```.`s.s`.v", cdf.schema)) self.assertTrue(verify_col_name("`m```.`s.s`.`v`", cdf.schema)) + def test_garbage_collection_checkpoint(self): + """ + SPARK-48258: Make sure garbage-collecting DataFrame remove the paired state + in Spark Connect server + """ + df = self.connect.range(10).localCheckpoint() + self.assertIsNotNone(df._cached_remote_relation_id) + cached_remote_relation_id = df._cached_remote_relation_id + + jvm = self.spark._jvm + session_holder = getattr( + getattr( + jvm.org.apache.spark.sql.connect.service, + "SparkConnectService$", + ), + "MODULE$", + ).getOrCreateIsolatedSession(self.connect.client._user_id, self.connect.client._session_id) + + # Check the state exists. + self.assertIsNotNone( + session_holder.dataFrameCache().getOrDefault(cached_remote_relation_id, None) + ) + + del df + + time.sleep(3) # Make sure removing is triggered, and executed in the server. + # Check the state was removed up on garbage-collection. + self.assertIsNone( + session_holder.dataFrameCache().getOrDefault(cached_remote_relation_id, None) + ) + if __name__ == "__main__": from pyspark.sql.tests.connect.test_connect_basic import * # noqa: F401 diff --git a/python/pyspark/sql/tests/connect/test_connect_error.py b/python/pyspark/sql/tests/connect/test_connect_error.py index 1297e62bb96f7..8c2dd71c0c6ab 100644 --- a/python/pyspark/sql/tests/connect/test_connect_error.py +++ b/python/pyspark/sql/tests/connect/test_connect_error.py @@ -158,12 +158,10 @@ def test_different_spark_session_join_or_union(self): def test_unsupported_functions(self): # SPARK-41225: Disable unsupported functions. df = self.connect.read.table(self.tbl_name) - for f in ( - "checkpoint", - "localCheckpoint", - ): - with self.assertRaises(NotImplementedError): - getattr(df, f)() + with self.assertRaises(NotImplementedError): + df.toJSON() + with self.assertRaises(NotImplementedError): + df.rdd def test_unsupported_jvm_attribute(self): # Unsupported jvm attributes for Spark session. diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index f491b496ddae5..ac46da587495a 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -844,6 +844,11 @@ def test_union_classmethod_usage(self): def test_isinstance_dataframe(self): self.assertIsInstance(self.spark.range(1), DataFrame) + def test_checkpoint_dataframe(self): + with io.StringIO() as buf, redirect_stdout(buf): + self.spark.range(1).localCheckpoint().explain() + self.assertIn("ExistingRDD", buf.getvalue()) + class DataFrameTests(DataFrameTestsMixin, ReusedSQLTestCase): pass