diff --git a/python/pyspark/sql/connect/client/artifact.py b/python/pyspark/sql/connect/client/artifact.py index ac33233a00ff..72a6ffa8bf68 100644 --- a/python/pyspark/sql/connect/client/artifact.py +++ b/python/pyspark/sql/connect/client/artifact.py @@ -427,6 +427,30 @@ def is_cached_artifact(self, hash: str) -> bool: status = resp.statuses.get(artifactName) return status.exists if status is not None else False + def get_cached_artifacts(self, hashes: list[str]) -> set[str]: + """ + Batch check which artifacts are already cached on the server. + Returns a set of hashes that are already cached. + """ + if not hashes: + return set() + + artifact_names = [f"{CACHE_PREFIX}/{hash}" for hash in hashes] + request = proto.ArtifactStatusesRequest( + user_context=self._user_context, session_id=self._session_id, names=artifact_names + ) + resp: proto.ArtifactStatusesResponse = self._stub.ArtifactStatus( + request, metadata=self._metadata + ) + + cached = set() + for hash in hashes: + artifact_name = f"{CACHE_PREFIX}/{hash}" + status = resp.statuses.get(artifact_name) + if status is not None and status.exists: + cached.add(hash) + return cached + def cache_artifact(self, blob: bytes) -> str: """ Cache the give blob at the session. @@ -442,3 +466,34 @@ def cache_artifact(self, blob: bytes) -> str: # TODO(SPARK-42658): Handle responses containing CRC failures. return hash + + def cache_artifacts(self, blobs: list[bytes]) -> list[str]: + """ + Cache the given blobs at the session. + + This method batches artifact status checks and uploads to minimize RPC overhead. + """ + # Compute hashes for all blobs upfront + hashes = [hashlib.sha256(blob).hexdigest() for blob in blobs] + unique_hashes = list(set(hashes)) + + # Batch check which artifacts are already cached + cached_hashes = self.get_cached_artifacts(unique_hashes) + + # Collect unique artifacts that need to be uploaded + seen_hashes = set() + artifacts_to_add = [] + for blob, hash in zip(blobs, hashes): + if hash not in cached_hashes and hash not in seen_hashes: + artifacts_to_add.append(new_cache_artifact(hash, InMemory(blob))) + seen_hashes.add(hash) + + # Batch upload all missing artifacts in a single RPC call + if artifacts_to_add: + requests = self._add_artifacts(artifacts_to_add) + response: proto.AddArtifactsResponse = self._retrieve_responses(requests) + summaries: List[proto.AddArtifactsResponse.ArtifactSummary] = [] + for summary in response.artifacts: + summaries.append(summary) + # TODO(SPARK-42658): Handle responses containing CRC failures. + return hashes diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index d0d191dbd7fd..414781d67cd4 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -2003,6 +2003,12 @@ def cache_artifact(self, blob: bytes) -> str: return self._artifact_manager.cache_artifact(blob) raise SparkConnectException("Invalid state during retry exception handling.") + def cache_artifacts(self, blobs: list[bytes]) -> list[str]: + for attempt in self._retrying(): + with attempt: + return self._artifact_manager.cache_artifacts(blobs) + raise SparkConnectException("Invalid state during retry exception handling.") + def _verify_response_integrity( self, response: Union[ diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index c5b6f5430d6d..82a6326c7dc5 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -429,16 +429,78 @@ def __init__( def plan(self, session: "SparkConnectClient") -> proto.Relation: plan = self._create_proto_relation() if self._table is not None: - sink = pa.BufferOutputStream() - with pa.ipc.new_stream(sink, self._table.schema) as writer: - for b in self._table.to_batches(): - writer.write_batch(b) - plan.local_relation.data = sink.getvalue().to_pybytes() + plan.local_relation.data = self._serialize_table() if self._schema is not None: plan.local_relation.schema = self._schema return plan + def _serialize_table(self) -> bytes: + assert self._table is not None + sink = pa.BufferOutputStream() + with pa.ipc.new_stream(sink, self._table.schema) as writer: + batches = self._table.to_batches() + for b in batches: + writer.write_batch(b) + return sink.getvalue().to_pybytes() + + def _serialize_table_chunks( + self, + max_chunk_size_rows: int, + max_chunk_size_bytes: int, + ) -> list[bytes]: + """ + Serialize the table into multiple chunks, each up to max_chunk_size_bytes bytes + and max_chunk_size_rows rows. + Each chunk is a valid Arrow IPC stream. + + This method processes the table in fixed-size batches (1024 rows) for + efficiency, matching the Scala implementation's batchSizeCheckInterval. + """ + assert self._table is not None + chunks = [] + schema = self._table.schema + + # Calculate schema serialization size once + schema_buffer = pa.BufferOutputStream() + with pa.ipc.new_stream(schema_buffer, schema): + pass # Just write schema + schema_size = len(schema_buffer.getvalue()) + + current_batches: list[pa.RecordBatch] = [] + current_size = schema_size + + for batch in self._table.to_batches(max_chunksize=min(1024, max_chunk_size_rows)): + batch_size = sum(arr.nbytes for arr in batch.columns) + + # If this batch would exceed limit and we have data, flush current chunk + if current_size > schema_size and current_size + batch_size > max_chunk_size_bytes: + combined = pa.Table.from_batches(current_batches, schema=schema) + sink = pa.BufferOutputStream() + with pa.ipc.new_stream(sink, schema) as writer: + writer.write_table(combined) + chunks.append(sink.getvalue().to_pybytes()) + current_batches = [] + current_size = schema_size + + current_batches.append(batch) + current_size += batch_size + + # Flush remaining batches + if current_batches: + combined = pa.Table.from_batches(current_batches, schema=schema) + sink = pa.BufferOutputStream() + with pa.ipc.new_stream(sink, schema) as writer: + writer.write_table(combined) + chunks.append(sink.getvalue().to_pybytes()) + + return chunks + + def _serialize_schema(self) -> bytes: + # the server uses UTF-8 for decoding the schema + assert self._schema is not None + return self._schema.encode("utf-8") + def serialize(self, session: "SparkConnectClient") -> bytes: p = self.plan(session) return bytes(p.local_relation.SerializeToString()) @@ -454,29 +516,34 @@ def _repr_html_(self) -> str: """ -class CachedLocalRelation(LogicalPlan): +class ChunkedCachedLocalRelation(LogicalPlan): """Creates a CachedLocalRelation plan object based on a hash of a LocalRelation.""" - def __init__(self, hash: str) -> None: + def __init__(self, data_hashes: list[str], schema_hash: Optional[str]) -> None: super().__init__(None) - self._hash = hash + self._data_hashes = data_hashes + self._schema_hash = schema_hash def plan(self, session: "SparkConnectClient") -> proto.Relation: plan = self._create_proto_relation() - clr = plan.cached_local_relation + clr = plan.chunked_cached_local_relation - clr.hash = self._hash + # Add hex string hashes directly to protobuf + for data_hash in self._data_hashes: + clr.dataHashes.append(data_hash) + if self._schema_hash is not None: + clr.schemaHash = self._schema_hash return plan def print(self, indent: int = 0) -> str: - return f"{' ' * indent}\n" + return f"{' ' * indent}\n" def _repr_html_(self) -> str: return """ """ diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 3774bcbdbfb0..4f057046cd43 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xe4\x03\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12P\n\x15model_summary_dataset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x01R\x13modelSummaryDataset\x88\x01\x01\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_typeB\x18\n\x16_model_summary_dataset"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\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"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\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\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\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\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xd2\x06\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x12\x65\n\x19transform_with_state_info\x18\x0b \x01(\x0b\x32%.spark.connect.TransformWithStateInfoH\x04R\x16transformWithStateInfo\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schemaB\x1c\n\x1a_transform_with_state_info"\xdf\x01\n\x16TransformWithStateInfo\x12\x1b\n\ttime_mode\x18\x01 \x01(\tR\x08timeMode\x12\x38\n\x16\x65vent_time_column_name\x18\x02 \x01(\tH\x00R\x13\x65ventTimeColumnName\x88\x01\x01\x12\x41\n\routput_schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x01R\x0coutputSchema\x88\x01\x01\x42\x19\n\x17_event_time_column_nameB\x10\n\x0e_output_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x8c\x1e\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12n\n\x1d\x63hunked_cached_local_relation\x18- \x01(\x0b\x32).spark.connect.ChunkedCachedLocalRelationH\x00R\x1a\x63hunkedCachedLocalRelation\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xe4\x03\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12P\n\x15model_summary_dataset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationH\x01R\x13modelSummaryDataset\x88\x01\x01\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_typeB\x18\n\x16_model_summary_dataset"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\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"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\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\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\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\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"p\n\x1a\x43hunkedCachedLocalRelation\x12\x1e\n\ndataHashes\x18\x01 \x03(\tR\ndataHashes\x12#\n\nschemaHash\x18\x02 \x01(\tH\x00R\nschemaHash\x88\x01\x01\x42\r\n\x0b_schemaHash"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xd2\x06\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x12\x65\n\x19transform_with_state_info\x18\x0b \x01(\x0b\x32%.spark.connect.TransformWithStateInfoH\x04R\x16transformWithStateInfo\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schemaB\x1c\n\x1a_transform_with_state_info"\xdf\x01\n\x16TransformWithStateInfo\x12\x1b\n\ttime_mode\x18\x01 \x01(\tR\x08timeMode\x12\x38\n\x16\x65vent_time_column_name\x18\x02 \x01(\tH\x00R\x13\x65ventTimeColumnName\x88\x01\x01\x12\x41\n\routput_schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x01R\x0coutputSchema\x88\x01\x01\x42\x19\n\x17_event_time_column_nameB\x10\n\x0e_output_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -79,171 +79,173 @@ _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_RELATION"]._serialized_start = 224 - _globals["_RELATION"]._serialized_end = 3964 - _globals["_MLRELATION"]._serialized_start = 3967 - _globals["_MLRELATION"]._serialized_end = 4451 - _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4179 - _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4414 - _globals["_FETCH"]._serialized_start = 4454 - _globals["_FETCH"]._serialized_end = 4785 - _globals["_FETCH_METHOD"]._serialized_start = 4570 - _globals["_FETCH_METHOD"]._serialized_end = 4785 - _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4658 - _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4785 - _globals["_UNKNOWN"]._serialized_start = 4787 - _globals["_UNKNOWN"]._serialized_end = 4796 - _globals["_RELATIONCOMMON"]._serialized_start = 4799 - _globals["_RELATIONCOMMON"]._serialized_end = 4941 - _globals["_SQL"]._serialized_start = 4944 - _globals["_SQL"]._serialized_end = 5422 - _globals["_SQL_ARGSENTRY"]._serialized_start = 5238 - _globals["_SQL_ARGSENTRY"]._serialized_end = 5328 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5330 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5422 - _globals["_WITHRELATIONS"]._serialized_start = 5424 - _globals["_WITHRELATIONS"]._serialized_end = 5541 - _globals["_READ"]._serialized_start = 5544 - _globals["_READ"]._serialized_end = 6207 - _globals["_READ_NAMEDTABLE"]._serialized_start = 5722 - _globals["_READ_NAMEDTABLE"]._serialized_end = 5914 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5856 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5914 - _globals["_READ_DATASOURCE"]._serialized_start = 5917 - _globals["_READ_DATASOURCE"]._serialized_end = 6194 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5856 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5914 - _globals["_PROJECT"]._serialized_start = 6209 - _globals["_PROJECT"]._serialized_end = 6326 - _globals["_FILTER"]._serialized_start = 6328 - _globals["_FILTER"]._serialized_end = 6440 - _globals["_JOIN"]._serialized_start = 6443 - _globals["_JOIN"]._serialized_end = 7104 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6782 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6874 - _globals["_JOIN_JOINTYPE"]._serialized_start = 6877 - _globals["_JOIN_JOINTYPE"]._serialized_end = 7085 - _globals["_SETOPERATION"]._serialized_start = 7107 - _globals["_SETOPERATION"]._serialized_end = 7586 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7423 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7537 - _globals["_LIMIT"]._serialized_start = 7588 - _globals["_LIMIT"]._serialized_end = 7664 - _globals["_OFFSET"]._serialized_start = 7666 - _globals["_OFFSET"]._serialized_end = 7745 - _globals["_TAIL"]._serialized_start = 7747 - _globals["_TAIL"]._serialized_end = 7822 - _globals["_AGGREGATE"]._serialized_start = 7825 - _globals["_AGGREGATE"]._serialized_end = 8591 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 8240 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 8351 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8353 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8429 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8432 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8591 - _globals["_SORT"]._serialized_start = 8594 - _globals["_SORT"]._serialized_end = 8754 - _globals["_DROP"]._serialized_start = 8757 - _globals["_DROP"]._serialized_end = 8898 - _globals["_DEDUPLICATE"]._serialized_start = 8901 - _globals["_DEDUPLICATE"]._serialized_end = 9141 - _globals["_LOCALRELATION"]._serialized_start = 9143 - _globals["_LOCALRELATION"]._serialized_end = 9232 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 9234 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 9306 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 9308 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 9363 - _globals["_SAMPLE"]._serialized_start = 9366 - _globals["_SAMPLE"]._serialized_end = 9639 - _globals["_RANGE"]._serialized_start = 9642 - _globals["_RANGE"]._serialized_end = 9787 - _globals["_SUBQUERYALIAS"]._serialized_start = 9789 - _globals["_SUBQUERYALIAS"]._serialized_end = 9903 - _globals["_REPARTITION"]._serialized_start = 9906 - _globals["_REPARTITION"]._serialized_end = 10048 - _globals["_SHOWSTRING"]._serialized_start = 10051 - _globals["_SHOWSTRING"]._serialized_end = 10193 - _globals["_HTMLSTRING"]._serialized_start = 10195 - _globals["_HTMLSTRING"]._serialized_end = 10309 - _globals["_STATSUMMARY"]._serialized_start = 10311 - _globals["_STATSUMMARY"]._serialized_end = 10403 - _globals["_STATDESCRIBE"]._serialized_start = 10405 - _globals["_STATDESCRIBE"]._serialized_end = 10486 - _globals["_STATCROSSTAB"]._serialized_start = 10488 - _globals["_STATCROSSTAB"]._serialized_end = 10589 - _globals["_STATCOV"]._serialized_start = 10591 - _globals["_STATCOV"]._serialized_end = 10687 - _globals["_STATCORR"]._serialized_start = 10690 - _globals["_STATCORR"]._serialized_end = 10827 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 10830 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10994 - _globals["_STATFREQITEMS"]._serialized_start = 10996 - _globals["_STATFREQITEMS"]._serialized_end = 11121 - _globals["_STATSAMPLEBY"]._serialized_start = 11124 - _globals["_STATSAMPLEBY"]._serialized_end = 11433 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11325 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11424 - _globals["_NAFILL"]._serialized_start = 11436 - _globals["_NAFILL"]._serialized_end = 11570 - _globals["_NADROP"]._serialized_start = 11573 - _globals["_NADROP"]._serialized_end = 11707 - _globals["_NAREPLACE"]._serialized_start = 11710 - _globals["_NAREPLACE"]._serialized_end = 12006 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11865 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 12006 - _globals["_TODF"]._serialized_start = 12008 - _globals["_TODF"]._serialized_end = 12096 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 12099 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12481 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12343 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12410 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12412 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12481 - _globals["_WITHCOLUMNS"]._serialized_start = 12483 - _globals["_WITHCOLUMNS"]._serialized_end = 12602 - _globals["_WITHWATERMARK"]._serialized_start = 12605 - _globals["_WITHWATERMARK"]._serialized_end = 12739 - _globals["_HINT"]._serialized_start = 12742 - _globals["_HINT"]._serialized_end = 12874 - _globals["_UNPIVOT"]._serialized_start = 12877 - _globals["_UNPIVOT"]._serialized_end = 13204 - _globals["_UNPIVOT_VALUES"]._serialized_start = 13134 - _globals["_UNPIVOT_VALUES"]._serialized_end = 13193 - _globals["_TRANSPOSE"]._serialized_start = 13206 - _globals["_TRANSPOSE"]._serialized_end = 13328 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13330 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13455 - _globals["_TOSCHEMA"]._serialized_start = 13457 - _globals["_TOSCHEMA"]._serialized_end = 13563 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13566 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13769 - _globals["_MAPPARTITIONS"]._serialized_start = 13772 - _globals["_MAPPARTITIONS"]._serialized_end = 14004 - _globals["_GROUPMAP"]._serialized_start = 14007 - _globals["_GROUPMAP"]._serialized_end = 14857 - _globals["_TRANSFORMWITHSTATEINFO"]._serialized_start = 14860 - _globals["_TRANSFORMWITHSTATEINFO"]._serialized_end = 15083 - _globals["_COGROUPMAP"]._serialized_start = 15086 - _globals["_COGROUPMAP"]._serialized_end = 15612 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15615 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15972 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15975 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16219 - _globals["_PYTHONUDTF"]._serialized_start = 16222 - _globals["_PYTHONUDTF"]._serialized_end = 16399 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16402 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16553 - _globals["_PYTHONDATASOURCE"]._serialized_start = 16555 - _globals["_PYTHONDATASOURCE"]._serialized_end = 16630 - _globals["_COLLECTMETRICS"]._serialized_start = 16633 - _globals["_COLLECTMETRICS"]._serialized_end = 16769 - _globals["_PARSE"]._serialized_start = 16772 - _globals["_PARSE"]._serialized_end = 17160 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5856 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5914 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17061 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17149 - _globals["_ASOFJOIN"]._serialized_start = 17163 - _globals["_ASOFJOIN"]._serialized_end = 17638 - _globals["_LATERALJOIN"]._serialized_start = 17641 - _globals["_LATERALJOIN"]._serialized_end = 17871 + _globals["_RELATION"]._serialized_end = 4076 + _globals["_MLRELATION"]._serialized_start = 4079 + _globals["_MLRELATION"]._serialized_end = 4563 + _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4291 + _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4526 + _globals["_FETCH"]._serialized_start = 4566 + _globals["_FETCH"]._serialized_end = 4897 + _globals["_FETCH_METHOD"]._serialized_start = 4682 + _globals["_FETCH_METHOD"]._serialized_end = 4897 + _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4770 + _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4897 + _globals["_UNKNOWN"]._serialized_start = 4899 + _globals["_UNKNOWN"]._serialized_end = 4908 + _globals["_RELATIONCOMMON"]._serialized_start = 4911 + _globals["_RELATIONCOMMON"]._serialized_end = 5053 + _globals["_SQL"]._serialized_start = 5056 + _globals["_SQL"]._serialized_end = 5534 + _globals["_SQL_ARGSENTRY"]._serialized_start = 5350 + _globals["_SQL_ARGSENTRY"]._serialized_end = 5440 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5442 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5534 + _globals["_WITHRELATIONS"]._serialized_start = 5536 + _globals["_WITHRELATIONS"]._serialized_end = 5653 + _globals["_READ"]._serialized_start = 5656 + _globals["_READ"]._serialized_end = 6319 + _globals["_READ_NAMEDTABLE"]._serialized_start = 5834 + _globals["_READ_NAMEDTABLE"]._serialized_end = 6026 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5968 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 6026 + _globals["_READ_DATASOURCE"]._serialized_start = 6029 + _globals["_READ_DATASOURCE"]._serialized_end = 6306 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5968 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 6026 + _globals["_PROJECT"]._serialized_start = 6321 + _globals["_PROJECT"]._serialized_end = 6438 + _globals["_FILTER"]._serialized_start = 6440 + _globals["_FILTER"]._serialized_end = 6552 + _globals["_JOIN"]._serialized_start = 6555 + _globals["_JOIN"]._serialized_end = 7216 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6894 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6986 + _globals["_JOIN_JOINTYPE"]._serialized_start = 6989 + _globals["_JOIN_JOINTYPE"]._serialized_end = 7197 + _globals["_SETOPERATION"]._serialized_start = 7219 + _globals["_SETOPERATION"]._serialized_end = 7698 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7535 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7649 + _globals["_LIMIT"]._serialized_start = 7700 + _globals["_LIMIT"]._serialized_end = 7776 + _globals["_OFFSET"]._serialized_start = 7778 + _globals["_OFFSET"]._serialized_end = 7857 + _globals["_TAIL"]._serialized_start = 7859 + _globals["_TAIL"]._serialized_end = 7934 + _globals["_AGGREGATE"]._serialized_start = 7937 + _globals["_AGGREGATE"]._serialized_end = 8703 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 8352 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 8463 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8465 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8541 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8544 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8703 + _globals["_SORT"]._serialized_start = 8706 + _globals["_SORT"]._serialized_end = 8866 + _globals["_DROP"]._serialized_start = 8869 + _globals["_DROP"]._serialized_end = 9010 + _globals["_DEDUPLICATE"]._serialized_start = 9013 + _globals["_DEDUPLICATE"]._serialized_end = 9253 + _globals["_LOCALRELATION"]._serialized_start = 9255 + _globals["_LOCALRELATION"]._serialized_end = 9344 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 9346 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 9418 + _globals["_CHUNKEDCACHEDLOCALRELATION"]._serialized_start = 9420 + _globals["_CHUNKEDCACHEDLOCALRELATION"]._serialized_end = 9532 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 9534 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 9589 + _globals["_SAMPLE"]._serialized_start = 9592 + _globals["_SAMPLE"]._serialized_end = 9865 + _globals["_RANGE"]._serialized_start = 9868 + _globals["_RANGE"]._serialized_end = 10013 + _globals["_SUBQUERYALIAS"]._serialized_start = 10015 + _globals["_SUBQUERYALIAS"]._serialized_end = 10129 + _globals["_REPARTITION"]._serialized_start = 10132 + _globals["_REPARTITION"]._serialized_end = 10274 + _globals["_SHOWSTRING"]._serialized_start = 10277 + _globals["_SHOWSTRING"]._serialized_end = 10419 + _globals["_HTMLSTRING"]._serialized_start = 10421 + _globals["_HTMLSTRING"]._serialized_end = 10535 + _globals["_STATSUMMARY"]._serialized_start = 10537 + _globals["_STATSUMMARY"]._serialized_end = 10629 + _globals["_STATDESCRIBE"]._serialized_start = 10631 + _globals["_STATDESCRIBE"]._serialized_end = 10712 + _globals["_STATCROSSTAB"]._serialized_start = 10714 + _globals["_STATCROSSTAB"]._serialized_end = 10815 + _globals["_STATCOV"]._serialized_start = 10817 + _globals["_STATCOV"]._serialized_end = 10913 + _globals["_STATCORR"]._serialized_start = 10916 + _globals["_STATCORR"]._serialized_end = 11053 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 11056 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 11220 + _globals["_STATFREQITEMS"]._serialized_start = 11222 + _globals["_STATFREQITEMS"]._serialized_end = 11347 + _globals["_STATSAMPLEBY"]._serialized_start = 11350 + _globals["_STATSAMPLEBY"]._serialized_end = 11659 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11551 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11650 + _globals["_NAFILL"]._serialized_start = 11662 + _globals["_NAFILL"]._serialized_end = 11796 + _globals["_NADROP"]._serialized_start = 11799 + _globals["_NADROP"]._serialized_end = 11933 + _globals["_NAREPLACE"]._serialized_start = 11936 + _globals["_NAREPLACE"]._serialized_end = 12232 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 12091 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 12232 + _globals["_TODF"]._serialized_start = 12234 + _globals["_TODF"]._serialized_end = 12322 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 12325 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12707 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12569 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12636 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12638 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12707 + _globals["_WITHCOLUMNS"]._serialized_start = 12709 + _globals["_WITHCOLUMNS"]._serialized_end = 12828 + _globals["_WITHWATERMARK"]._serialized_start = 12831 + _globals["_WITHWATERMARK"]._serialized_end = 12965 + _globals["_HINT"]._serialized_start = 12968 + _globals["_HINT"]._serialized_end = 13100 + _globals["_UNPIVOT"]._serialized_start = 13103 + _globals["_UNPIVOT"]._serialized_end = 13430 + _globals["_UNPIVOT_VALUES"]._serialized_start = 13360 + _globals["_UNPIVOT_VALUES"]._serialized_end = 13419 + _globals["_TRANSPOSE"]._serialized_start = 13432 + _globals["_TRANSPOSE"]._serialized_end = 13554 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13556 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13681 + _globals["_TOSCHEMA"]._serialized_start = 13683 + _globals["_TOSCHEMA"]._serialized_end = 13789 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13792 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13995 + _globals["_MAPPARTITIONS"]._serialized_start = 13998 + _globals["_MAPPARTITIONS"]._serialized_end = 14230 + _globals["_GROUPMAP"]._serialized_start = 14233 + _globals["_GROUPMAP"]._serialized_end = 15083 + _globals["_TRANSFORMWITHSTATEINFO"]._serialized_start = 15086 + _globals["_TRANSFORMWITHSTATEINFO"]._serialized_end = 15309 + _globals["_COGROUPMAP"]._serialized_start = 15312 + _globals["_COGROUPMAP"]._serialized_end = 15838 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15841 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 16198 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 16201 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16445 + _globals["_PYTHONUDTF"]._serialized_start = 16448 + _globals["_PYTHONUDTF"]._serialized_end = 16625 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16628 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16779 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16781 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16856 + _globals["_COLLECTMETRICS"]._serialized_start = 16859 + _globals["_COLLECTMETRICS"]._serialized_end = 16995 + _globals["_PARSE"]._serialized_start = 16998 + _globals["_PARSE"]._serialized_end = 17386 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5968 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 6026 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17287 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17375 + _globals["_ASOFJOIN"]._serialized_start = 17389 + _globals["_ASOFJOIN"]._serialized_end = 17864 + _globals["_LATERALJOIN"]._serialized_start = 17867 + _globals["_LATERALJOIN"]._serialized_end = 18097 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index e1eb7945c19f..c6f20c158a6c 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -108,6 +108,7 @@ class Relation(google.protobuf.message.Message): TRANSPOSE_FIELD_NUMBER: builtins.int UNRESOLVED_TABLE_VALUED_FUNCTION_FIELD_NUMBER: builtins.int LATERAL_JOIN_FIELD_NUMBER: builtins.int + CHUNKED_CACHED_LOCAL_RELATION_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -216,6 +217,8 @@ class Relation(google.protobuf.message.Message): @property def lateral_join(self) -> global___LateralJoin: ... @property + def chunked_cached_local_relation(self) -> global___ChunkedCachedLocalRelation: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -301,6 +304,7 @@ class Relation(google.protobuf.message.Message): transpose: global___Transpose | None = ..., unresolved_table_valued_function: global___UnresolvedTableValuedFunction | None = ..., lateral_join: global___LateralJoin | None = ..., + chunked_cached_local_relation: global___ChunkedCachedLocalRelation | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -334,6 +338,8 @@ class Relation(google.protobuf.message.Message): b"cached_remote_relation", "catalog", b"catalog", + "chunked_cached_local_relation", + b"chunked_cached_local_relation", "co_group_map", b"co_group_map", "collect_metrics", @@ -459,6 +465,8 @@ class Relation(google.protobuf.message.Message): b"cached_remote_relation", "catalog", b"catalog", + "chunked_cached_local_relation", + b"chunked_cached_local_relation", "co_group_map", b"co_group_map", "collect_metrics", @@ -614,6 +622,7 @@ class Relation(google.protobuf.message.Message): "transpose", "unresolved_table_valued_function", "lateral_join", + "chunked_cached_local_relation", "fill_na", "drop_na", "replace", @@ -2084,7 +2093,9 @@ class LocalRelation(google.protobuf.message.Message): global___LocalRelation = LocalRelation class CachedLocalRelation(google.protobuf.message.Message): - """A local relation that has been cached already.""" + """A local relation that has been cached already. + CachedLocalRelation doesn't support LocalRelations of size over 2GB. + """ DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2100,6 +2111,51 @@ class CachedLocalRelation(google.protobuf.message.Message): global___CachedLocalRelation = CachedLocalRelation +class ChunkedCachedLocalRelation(google.protobuf.message.Message): + """A local relation that has been cached already.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + DATAHASHES_FIELD_NUMBER: builtins.int + SCHEMAHASH_FIELD_NUMBER: builtins.int + @property + def dataHashes( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: + """(Required) A list of sha-256 hashes for representing LocalRelation.data. + Data is serialized in Arrow IPC streaming format, each batch is cached on the server as + a separate artifact. Each hash represents one batch stored on the server. + Hashes are hex-encoded strings (e.g., "a3b2c1d4..."). + """ + schemaHash: builtins.str + """(Optional) A sha-256 hash of the serialized LocalRelation.schema. + Scala clients always provide the schema, Python clients can omit it. + Hash is a hex-encoded string (e.g., "a3b2c1d4..."). + """ + def __init__( + self, + *, + dataHashes: collections.abc.Iterable[builtins.str] | None = ..., + schemaHash: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_schemaHash", b"_schemaHash", "schemaHash", b"schemaHash" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_schemaHash", b"_schemaHash", "dataHashes", b"dataHashes", "schemaHash", b"schemaHash" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_schemaHash", b"_schemaHash"] + ) -> typing_extensions.Literal["schemaHash"] | None: ... + +global___ChunkedCachedLocalRelation = ChunkedCachedLocalRelation + class CachedRemoteRelation(google.protobuf.message.Message): """Represents a remote relation that has been cached on server.""" diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index f759137fac1d..2a678c95c925 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -61,7 +61,7 @@ Range, LocalRelation, LogicalPlan, - CachedLocalRelation, + ChunkedCachedLocalRelation, CachedRelation, CachedRemoteRelation, SubqueryAlias, @@ -535,6 +535,8 @@ def createDataFrame( "spark.sql.timestampType", "spark.sql.session.timeZone", "spark.sql.session.localRelationCacheThreshold", + "spark.sql.session.localRelationChunkSizeRows", + "spark.sql.session.localRelationChunkSizeBytes", "spark.sql.execution.pandas.convertToArrowArraySafely", "spark.sql.execution.pandas.inferPandasDictAsMap", "spark.sql.pyspark.inferNestedDictAsStruct.enabled", @@ -755,10 +757,21 @@ def createDataFrame( else: local_relation = LocalRelation(_table) - cache_threshold = configs["spark.sql.session.localRelationCacheThreshold"] + # get_config_dict throws [SQL_CONF_NOT_FOUND] if the key is not found. + cache_threshold = int( + configs["spark.sql.session.localRelationCacheThreshold"] # type: ignore[arg-type] + ) + max_chunk_size_rows = int( + configs["spark.sql.session.localRelationChunkSizeRows"] # type: ignore[arg-type] + ) + max_chunk_size_bytes = int( + configs["spark.sql.session.localRelationChunkSizeBytes"] # type: ignore[arg-type] + ) plan: LogicalPlan = local_relation - if cache_threshold is not None and int(cache_threshold) <= _table.nbytes: - plan = CachedLocalRelation(self._cache_local_relation(local_relation)) + if cache_threshold <= _table.nbytes: + plan = self._cache_local_relation( + local_relation, max_chunk_size_rows, max_chunk_size_bytes + ) df = DataFrame(plan, self) if _cols is not None and len(_cols) > 0: @@ -1031,12 +1044,36 @@ def addArtifacts( addArtifact = addArtifacts - def _cache_local_relation(self, local_relation: LocalRelation) -> str: + def _cache_local_relation( + self, + local_relation: LocalRelation, + max_chunk_size_rows: int, + max_chunk_size_bytes: int, + ) -> ChunkedCachedLocalRelation: """ Cache the local relation at the server side if it has not been cached yet. + + Should only be called on LocalRelations with _table set. """ - serialized = local_relation.serialize(self._client) - return self._client.cache_artifact(serialized) + assert local_relation._table is not None + has_schema = local_relation._schema is not None + + # Serialize table into chunks + data_chunks = local_relation._serialize_table_chunks( + max_chunk_size_rows, max_chunk_size_bytes + ) + blobs = data_chunks.copy() # Start with data chunks + + if has_schema: + blobs.append(local_relation._serialize_schema()) + + hashes = self._client.cache_artifacts(blobs) + + # Extract data hashes and schema hash + data_hashes = hashes[: len(data_chunks)] + schema_hash = hashes[len(data_chunks)] if has_schema else None + + return ChunkedCachedLocalRelation(data_hashes, schema_hash) def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: if urllib.parse.urlparse(dest_path).scheme: diff --git a/python/pyspark/sql/tests/arrow/test_arrow.py b/python/pyspark/sql/tests/arrow/test_arrow.py index 819639c63a2c..be7dd2febc94 100644 --- a/python/pyspark/sql/tests/arrow/test_arrow.py +++ b/python/pyspark/sql/tests/arrow/test_arrow.py @@ -420,6 +420,29 @@ def check_toPandas_respect_session_timezone(self, arrow_enabled): ) assert_frame_equal(pdf_ny, pdf_la_corrected) + def check_cached_local_relation_changing_values(self): + import random + import string + + row_size = 1000 + row_count = 64 * 1000 + suffix = "abcdef" + str_value = ( + "".join(random.choices(string.ascii_letters + string.digits, k=row_size)) + suffix + ) + data = [(i, str_value) for i in range(row_count)] + + for _ in range(2): + df = self.spark.createDataFrame(data, ["col1", "col2"]) + assert df.count() == row_count + assert not df.filter(df["col2"].endswith(suffix)).isEmpty() + + def check_large_cached_local_relation_same_values(self): + data = [("C000000032", "R20", 0.2555)] * 500_000 + pdf = pd.DataFrame(data=data, columns=["Contrat", "Recommandation", "Distance"]) + df = self.spark.createDataFrame(pdf) + df.collect() + def test_toArrow_keep_utc_timezone(self): df = self.spark.createDataFrame(self.data, schema=self.schema) diff --git a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py index fa8cf286b9bd..2cc089a7c0d5 100644 --- a/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py +++ b/python/pyspark/sql/tests/connect/arrow/test_parity_arrow.py @@ -78,6 +78,12 @@ def test_createDataFrame_pandas_respect_session_timezone(self): def test_toPandas_respect_session_timezone(self): self.check_toPandas_respect_session_timezone(True) + def test_cached_local_relation_changing_values(self): + self.check_cached_local_relation_changing_values() + + def test_large_cached_local_relation_same_values(self): + self.check_large_cached_local_relation_same_values() + def test_toPandas_with_array_type(self): self.check_toPandas_with_array_type(True) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala index 9a69c3d2488f..f715f8f9ed8c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConf.scala @@ -61,9 +61,12 @@ private[sql] object SqlApiConf { val SESSION_LOCAL_TIMEZONE_KEY: String = SqlApiConfHelper.SESSION_LOCAL_TIMEZONE_KEY val ARROW_EXECUTION_USE_LARGE_VAR_TYPES: String = SqlApiConfHelper.ARROW_EXECUTION_USE_LARGE_VAR_TYPES - val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = { + val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = SqlApiConfHelper.LOCAL_RELATION_CACHE_THRESHOLD_KEY - } + val LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY: String = + SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY + val LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY: String = + SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String = SqlApiConfHelper.PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY val PARSER_DFA_CACHE_FLUSH_RATIO_KEY: String = diff --git a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala index 727620bd5bd0..b839caba3f54 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/internal/SqlApiConfHelper.scala @@ -32,6 +32,9 @@ private[sql] object SqlApiConfHelper { val CASE_SENSITIVE_KEY: String = "spark.sql.caseSensitive" val SESSION_LOCAL_TIMEZONE_KEY: String = "spark.sql.session.timeZone" val LOCAL_RELATION_CACHE_THRESHOLD_KEY: String = "spark.sql.session.localRelationCacheThreshold" + val LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY: String = "spark.sql.session.localRelationChunkSizeRows" + val LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY: String = + "spark.sql.session.localRelationChunkSizeBytes" val ARROW_EXECUTION_USE_LARGE_VAR_TYPES = "spark.sql.execution.arrow.useLargeVarTypes" val PARSER_DFA_CACHE_FLUSH_THRESHOLD_KEY: String = "spark.sql.parser.parserDfaCacheFlushThreshold" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 0c0ed490f973..7096b8e46055 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -5894,7 +5894,47 @@ object SQLConf { .version("3.5.0") .intConf .checkValue(_ >= 0, "The threshold of cached local relations must not be negative") - .createWithDefault(64 * 1024 * 1024) + .createWithDefault(1024 * 1024) + + val LOCAL_RELATION_CHUNK_SIZE_ROWS = + buildConf(SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY) + .doc("The chunk size in number of rows when splitting ChunkedCachedLocalRelation.data " + + "into batches. A new chunk is created when either " + + "spark.sql.session.localRelationChunkSizeBytes " + + "or spark.sql.session.localRelationChunkSizeRows is reached.") + .version("4.1.0") + .intConf + .checkValue(_ > 0, "The chunk size in number of rows must be positive") + .createWithDefault(10000) + + val LOCAL_RELATION_CHUNK_SIZE_BYTES = + buildConf(SqlApiConfHelper.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY) + .doc("The chunk size in bytes when splitting ChunkedCachedLocalRelation.data " + + "into batches. A new chunk is created when either " + + "spark.sql.session.localRelationChunkSizeBytes " + + "or spark.sql.session.localRelationChunkSizeRows is reached.") + .version("4.1.0") + .longConf + .checkValue(_ > 0, "The chunk size in bytes must be positive") + .createWithDefault(16 * 1024 * 1024L) + + val LOCAL_RELATION_CHUNK_SIZE_LIMIT = + buildConf("spark.sql.session.localRelationChunkSizeLimit") + .internal() + .doc("Limit on how large a single chunk of a ChunkedCachedLocalRelation.data " + + "can be in bytes. If the limit is exceeded, an exception is thrown.") + .version("4.1.0") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("2000MB") + + val LOCAL_RELATION_SIZE_LIMIT = + buildConf("spark.sql.session.localRelationSizeLimit") + .internal() + .doc("Limit on how large ChunkedCachedLocalRelation.data can be in bytes." + + "If the limit is exceeded, an exception is thrown.") + .version("4.1.0") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("3GB") val DECORRELATE_JOIN_PREDICATE_ENABLED = buildConf("spark.sql.optimizer.decorrelateJoinPredicate.enabled") @@ -7140,6 +7180,10 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def rangeExchangeSampleSizePerPartition: Int = getConf(RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION) + def localRelationChunkSizeLimit: Long = getConf(LOCAL_RELATION_CHUNK_SIZE_LIMIT) + + def localRelationSizeLimit: Long = getConf(LOCAL_RELATION_SIZE_LIMIT) + def arrowPySparkEnabled: Boolean = getConf(ARROW_PYSPARK_EXECUTION_ENABLED) def arrowLocalRelationThreshold: Long = getConf(ARROW_LOCAL_RELATION_THRESHOLD) diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SparkSessionE2ESuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SparkSessionE2ESuite.scala index 4c0073cad567..6678a11a80b0 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SparkSessionE2ESuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/SparkSessionE2ESuite.scala @@ -450,4 +450,30 @@ class SparkSessionE2ESuite extends ConnectFunSuite with RemoteSparkSession { Map("one" -> "1", "two" -> "2")) assert(df.as(StringEncoder).collect().toSet == Set("one", "two")) } + + test("dataframes with cached local relations succeed - changing values") { + val rowSize = 1000 + val rowCount = 64 * 1000 + val suffix = "abcdef" + val str = scala.util.Random.alphanumeric.take(rowSize).mkString + suffix + val data = Seq.tabulate(rowCount)(i => (i, str)) + for (_ <- 0 until 2) { + val df = spark.createDataFrame(data) + assert(df.count() === rowCount) + assert(!df.filter(df("_2").endsWith(suffix)).isEmpty) + } + } + + test("dataframes with cached local relations succeed - same values") { + val rowSize = 1000 + val rowCount = 64 * 1000 + val suffix = "abcdef" + val str = scala.util.Random.alphanumeric.take(rowSize).mkString + suffix + val data = Seq.tabulate(rowCount)(_ => (0, str)) + for (_ <- 0 until 2) { + val df = spark.createDataFrame(data) + assert(df.count() === rowCount) + assert(!df.filter(df("_2").endsWith(suffix)).isEmpty) + } + } } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index ccb674e812dc..1583785e69fb 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -80,6 +80,7 @@ message Relation { Transpose transpose = 42; UnresolvedTableValuedFunction unresolved_table_valued_function = 43; LateralJoin lateral_join = 44; + ChunkedCachedLocalRelation chunked_cached_local_relation = 45; // NA functions NAFill fill_na = 90; @@ -499,6 +500,7 @@ message LocalRelation { } // A local relation that has been cached already. +// CachedLocalRelation doesn't support LocalRelations of size over 2GB. message CachedLocalRelation { // `userId` and `sessionId` fields are deleted since the server must always use the active // session/user rather than arbitrary values provided by the client. It is never valid to access @@ -510,6 +512,20 @@ message CachedLocalRelation { string hash = 3; } +// A local relation that has been cached already. +message ChunkedCachedLocalRelation { + // (Required) A list of sha-256 hashes for representing LocalRelation.data. + // Data is serialized in Arrow IPC streaming format, each batch is cached on the server as + // a separate artifact. Each hash represents one batch stored on the server. + // Hashes are hex-encoded strings (e.g., "a3b2c1d4..."). + repeated string dataHashes = 1; + + // (Optional) A sha-256 hash of the serialized LocalRelation.schema. + // Scala clients always provide the schema, Python clients can omit it. + // Hash is a hex-encoded string (e.g., "a3b2c1d4..."). + optional string schemaHash = 2; +} + // Represents a remote relation that has been cached on server. message CachedRemoteRelation { // (Required) ID of the remote related (assigned by the service). diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index f7869a8b4dd8..0d9d4e5d60f0 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -31,6 +31,7 @@ import scala.reflect.runtime.universe.TypeTag import scala.util.Try import com.google.common.cache.{CacheBuilder, CacheLoader} +import com.google.protobuf.ByteString import io.grpc.ClientInterceptor import org.apache.arrow.memory.RootAllocator @@ -116,16 +117,40 @@ class SparkSession private[sql] ( private def createDataset[T](encoder: AgnosticEncoder[T], data: Iterator[T]): Dataset[T] = { newDataset(encoder) { builder => if (data.nonEmpty) { - val arrowData = - ArrowSerializer.serialize(data, encoder, allocator, timeZoneId, largeVarTypes) - if (arrowData.size() <= conf.get(SqlApiConf.LOCAL_RELATION_CACHE_THRESHOLD_KEY).toInt) { + val threshold = conf.get(SqlApiConf.LOCAL_RELATION_CACHE_THRESHOLD_KEY).toInt + val maxRecordsPerBatch = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_ROWS_KEY).toInt + val maxBatchSize = conf.get(SqlApiConf.LOCAL_RELATION_CHUNK_SIZE_BYTES_KEY).toInt + // Serialize with chunking support + val it = ArrowSerializer.serialize( + data, + encoder, + allocator, + maxRecordsPerBatch = maxRecordsPerBatch, + maxBatchSize = maxBatchSize, + timeZoneId = timeZoneId, + largeVarTypes = largeVarTypes, + batchSizeCheckInterval = math.min(1024, maxRecordsPerBatch)) + + val chunks = + try { + it.toArray + } finally { + it.close() + } + + // If we got multiple chunks or a single large chunk, use ChunkedCachedLocalRelation + val totalSize = chunks.map(_.length).sum + if (chunks.length > 1 || totalSize > threshold) { + val (dataHashes, schemaHash) = client.cacheLocalRelation(chunks, encoder.schema.json) + builder.getChunkedCachedLocalRelationBuilder + .setSchemaHash(schemaHash) + .addAllDataHashes(dataHashes.asJava) + } else { + // Small data, use LocalRelation directly + val arrowData = ByteString.copyFrom(chunks(0)) builder.getLocalRelationBuilder .setSchema(encoder.schema.json) .setData(arrowData) - } else { - val hash = client.cacheLocalRelation(arrowData, encoder.schema.json) - builder.getCachedLocalRelationBuilder - .setHash(hash) } } else { builder.getLocalRelationBuilder diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala index 213cd1d2e867..44a2a7aa9a2f 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala @@ -185,6 +185,38 @@ class ArtifactManager( } else false } + /** + * Batch check which artifacts are already cached on the server. Returns a Set of hashes that + * are already cached. + */ + private[client] def getCachedArtifacts(hashes: Seq[String]): Set[String] = { + if (hashes.isEmpty) { + return Set.empty + } + + val artifactNames = hashes.map(hash => s"${Artifact.CACHE_PREFIX}/$hash") + val request = proto.ArtifactStatusesRequest + .newBuilder() + .setUserContext(clientConfig.userContext) + .setClientType(clientConfig.userAgent) + .setSessionId(sessionId) + .addAllNames(artifactNames.asJava) + .build() + + val response = bstub.artifactStatus(request) + if (SparkStringUtils.isNotEmpty(response.getSessionId) && + response.getSessionId != sessionId) { + throw new IllegalStateException( + s"Session ID mismatch: $sessionId != ${response.getSessionId}") + } + + val statuses = response.getStatusesMap + hashes.filter { hash => + val artifactName = s"${Artifact.CACHE_PREFIX}/$hash" + statuses.containsKey(artifactName) && statuses.get(artifactName).getExists + }.toSet + } + /** * Cache the give blob at the session. */ @@ -196,6 +228,38 @@ class ArtifactManager( hash } + /** + * Cache the given blobs at the session. + * + * This method batches artifact status checks and uploads to minimize RPC overhead. Returns the + * list of hashes corresponding to the input blobs. + */ + def cacheArtifacts(blobs: Array[Array[Byte]]): Seq[String] = { + // Compute hashes for all blobs upfront + val hashes = blobs.map(sha256Hex).toSeq + val uniqueHashes = hashes.distinct + + // Batch check which artifacts are already cached + val cachedHashes = getCachedArtifacts(uniqueHashes) + + // Collect unique artifacts that need to be uploaded + val seenHashes = scala.collection.mutable.Set[String]() + val uniqueBlobsToUpload = scala.collection.mutable.ListBuffer[Artifact]() + for ((blob, hash) <- blobs.zip(hashes)) { + if (!cachedHashes.contains(hash) && !seenHashes.contains(hash)) { + uniqueBlobsToUpload += newCacheArtifact(hash, new Artifact.InMemory(blob)) + seenHashes.add(hash) + } + } + + // Batch upload all missing artifacts in a single RPC call + if (uniqueBlobsToUpload.nonEmpty) { + addArtifacts(uniqueBlobsToUpload.toList) + } + + hashes + } + /** * Upload all class file artifacts from the local REPL(s) to the server. * diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala index 3c328681dd9a..fa32eba91eb2 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala @@ -25,7 +25,6 @@ import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.util.Properties -import com.google.protobuf.ByteString import io.grpc._ import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} @@ -404,16 +403,23 @@ private[sql] class SparkConnectClient( } /** - * Cache the given local relation at the server, and return its key in the remote cache. + * Cache the given local relation Arrow stream from a local file and return its hashes. The file + * is streamed in chunks and does not need to fit in memory. + * + * This method batches artifact status checks and uploads to minimize RPC overhead. */ - private[sql] def cacheLocalRelation(data: ByteString, schema: String): String = { - val localRelation = proto.Relation - .newBuilder() - .getLocalRelationBuilder - .setSchema(schema) - .setData(data) - .build() - artifactManager.cacheArtifact(localRelation.toByteArray) + private[sql] def cacheLocalRelation( + data: Array[Array[Byte]], + schema: String): (Seq[String], String) = { + val schemaBytes = schema.getBytes + val allBlobs = data :+ schemaBytes + val allHashes = artifactManager.cacheArtifacts(allBlobs) + + // Last hash is the schema hash, rest are data hashes + val dataHashes = allHashes.dropRight(1) + val schemaHash = allHashes.last + + (dataHashes, schemaHash) } /** diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala index 0dd4192908b9..fcef696c88af 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/InvalidInputErrors.scala @@ -73,12 +73,32 @@ object InvalidInputErrors { s"Not found any cached local relation with the hash: " + s"$hash in the session with sessionUUID $sessionUUID.") + def notFoundChunkedCachedLocalRelationBlock( + hash: String, + sessionUUID: String): InvalidPlanInput = + InvalidPlanInput( + s"Not found chunked cached local relation block with the hash: " + + s"$hash in the session with sessionUUID $sessionUUID.") + + def localRelationSizeLimitExceeded(actualSize: Long, limit: Long): InvalidPlanInput = + InvalidPlanInput( + s"Cached local relation size ($actualSize bytes) exceeds the limit ($limit bytes).") + + def localRelationChunkSizeLimitExceeded(limit: Long): InvalidPlanInput = + InvalidPlanInput(s"One of cached local relation chunks exceeded the limit of $limit bytes.") + def withColumnsRequireSingleNamePart(got: String): InvalidPlanInput = InvalidPlanInput(s"WithColumns require column name only contains one name part, but got $got") def inputDataForLocalRelationNoSchema(): InvalidPlanInput = InvalidPlanInput("Input data for LocalRelation does not produce a schema.") + def chunkedCachedLocalRelationWithoutData(): InvalidPlanInput = + InvalidPlanInput("ChunkedCachedLocalRelation should contain data.") + + def chunkedCachedLocalRelationChunksWithDifferentSchema(): InvalidPlanInput = + InvalidPlanInput("ChunkedCachedLocalRelation data chunks have different schema.") + def schemaRequiredForLocalRelation(): InvalidPlanInput = InvalidPlanInput("Schema for LocalRelation is required when the input data is not provided.") diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 97bdf236e020..0824413343bc 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.connect.planner import java.util.{HashMap, Properties, UUID} +import scala.collection.immutable.ArraySeq import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.util.Try @@ -43,7 +44,7 @@ import org.apache.spark.internal.{Logging, LogKeys} import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{AnalysisException, Column, Encoders, ForeachWriter, Row} -import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} +import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LocalTempView, MultiAlias, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedOrdinal, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedStarWithColumns, UnresolvedStarWithColumnsRenames, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{ProductEncoder, RowEncoder => AgnosticRowEncoder, StringEncoder, UnboundRowEncoder} @@ -196,6 +197,8 @@ class SparkConnectPlanner( transformWithWatermark(rel.getWithWatermark) case proto.Relation.RelTypeCase.CACHED_LOCAL_RELATION => transformCachedLocalRelation(rel.getCachedLocalRelation) + case proto.Relation.RelTypeCase.CHUNKED_CACHED_LOCAL_RELATION => + transformChunkedCachedLocalRelation(rel.getChunkedCachedLocalRelation) case proto.Relation.RelTypeCase.HINT => transformHint(rel.getHint) case proto.Relation.RelTypeCase.UNPIVOT => transformUnpivot(rel.getUnpivot) case proto.Relation.RelTypeCase.TRANSPOSE => transformTranspose(rel.getTranspose) @@ -1483,25 +1486,128 @@ class SparkConnectPlanner( rel.getSchema, parseDatatypeString, fallbackParser = DataType.fromJson) - schema = schemaType match { - case s: StructType => s - case d => StructType(Seq(StructField("value", d))) - } + schema = toStructTypeOrWrap(schemaType) } if (rel.hasData) { val (rows, structType) = ArrowConverters.fromIPCStream(rel.getData.toByteArray, TaskContext.get()) - if (structType == null) { - throw InvalidInputErrors.inputDataForLocalRelationNoSchema() + buildLocalRelationFromRows(rows, structType, Option(schema)) + } else { + if (schema == null) { + throw InvalidInputErrors.schemaRequiredForLocalRelation() } - val attributes = DataTypeUtils.toAttributes(structType) - val proj = UnsafeProjection.create(attributes, attributes) - val data = rows.map(proj) + LocalRelation(schema) + } + } - if (schema == null) { - logical.LocalRelation(attributes, data.map(_.copy()).toSeq) + private def readChunkedCachedLocalRelationBlock(hash: String): Array[Byte] = { + val blockManager = session.sparkContext.env.blockManager + val blockId = CacheId(sessionHolder.session.sessionUUID, hash) + val bytes = blockManager.getLocalBytes(blockId) + bytes + .map { blockData => + try { + blockData.toInputStream().readAllBytes() + } finally { + blockManager.releaseLock(blockId) + } + } + .getOrElse { + throw InvalidInputErrors.notFoundChunkedCachedLocalRelationBlock( + blockId.hash, + blockId.sessionUUID) + } + } + + private def getBlockSize(hash: String): Long = { + val blockManager = session.sparkContext.env.blockManager + val blockId = CacheId(sessionHolder.session.sessionUUID, hash) + blockManager.getStatus(blockId).map(status => status.memSize + status.diskSize).getOrElse(0L) + } + + private def transformChunkedCachedLocalRelation( + rel: proto.ChunkedCachedLocalRelation): LogicalPlan = { + if (rel.getDataHashesCount == 0) { + throw InvalidInputErrors.chunkedCachedLocalRelationWithoutData() + } + val dataHashes = rel.getDataHashesList.asScala + val allHashes = dataHashes ++ ( + if (rel.hasSchemaHash) { + Seq(rel.getSchemaHash) } else { + Seq.empty + } + ) + val allSizes = allHashes.map(hash => getBlockSize(hash)) + val totalSize = allSizes.sum + + val relationSizeLimit = session.sessionState.conf.localRelationSizeLimit + val chunkSizeLimit = session.sessionState.conf.localRelationChunkSizeLimit + if (totalSize > relationSizeLimit) { + throw InvalidInputErrors.localRelationSizeLimitExceeded(totalSize, relationSizeLimit) + } + if (allSizes.exists(_ > chunkSizeLimit)) { + throw InvalidInputErrors.localRelationChunkSizeLimitExceeded(chunkSizeLimit) + } + + var schema: StructType = null + if (rel.hasSchemaHash) { + val schemaBytes = readChunkedCachedLocalRelationBlock(rel.getSchemaHash) + val schemaString = new String(schemaBytes) + val schemaType = DataType.parseTypeWithFallback( + schemaString, + parseDatatypeString, + fallbackParser = DataType.fromJson) + schema = toStructTypeOrWrap(schemaType) + } + + // Load and combine all batches + var combinedRows: Iterator[InternalRow] = Iterator.empty + var structType: StructType = null + + for ((dataHash, batchIndex) <- dataHashes.zipWithIndex) { + val dataBytes = readChunkedCachedLocalRelationBlock(dataHash) + val (batchRows, batchStructType) = + ArrowConverters.fromIPCStream(dataBytes, TaskContext.get()) + + // For the first batch, set the schema; for subsequent batches, verify compatibility + if (batchIndex == 0) { + structType = batchStructType + combinedRows = batchRows + + } else { + if (batchStructType != structType) { + throw InvalidInputErrors.chunkedCachedLocalRelationChunksWithDifferentSchema() + } + combinedRows = combinedRows ++ batchRows + } + } + + buildLocalRelationFromRows(combinedRows, structType, Option(schema)) + } + + private def toStructTypeOrWrap(dt: DataType): StructType = dt match { + case s: StructType => s + case d => StructType(Seq(StructField("value", d))) + } + + private def buildLocalRelationFromRows( + rows: Iterator[InternalRow], + structType: StructType, + schemaOpt: Option[StructType]): LogicalPlan = { + if (structType == null) { + throw InvalidInputErrors.inputDataForLocalRelationNoSchema() + } + + val attributes = DataTypeUtils.toAttributes(structType) + val initialProjection = UnsafeProjection.create(attributes, attributes) + val data = rows.map(initialProjection) + + schemaOpt match { + case None => + logical.LocalRelation(attributes, ArraySeq.unsafeWrapArray(data.map(_.copy()).toArray)) + case Some(schema) => def normalize(dt: DataType): DataType = dt match { case udt: UserDefinedType[_] => normalize(udt.sqlType) case StructType(fields) => @@ -1533,12 +1639,6 @@ class SparkConnectPlanner( logical.LocalRelation( DataTypeUtils.toAttributes(schema), data.map(proj).map(_.copy()).toSeq) - } - } else { - if (schema == null) { - throw InvalidInputErrors.schemaRequiredForLocalRelation() - } - LocalRelation(schema) } }