From 514ca811e52f90bfb866fd38090f807713a4f613 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Thu, 6 Apr 2023 00:39:42 -0700 Subject: [PATCH 01/12] Add spark connect for applyInPandasWithState --- .../protobuf/spark/connect/relations.proto | 17 ++ .../connect/planner/SparkConnectPlanner.scala | 21 ++ python/pyspark/sql/connect/group.py | 269 +++++++++++++++++- python/pyspark/sql/connect/plan.py | 40 +++ python/pyspark/sql/connect/proto/base_pb2.pyi | 57 ++++ .../pyspark/sql/connect/proto/catalog_pb2.pyi | 29 ++ .../sql/connect/proto/commands_pb2.pyi | 21 ++ .../pyspark/sql/connect/proto/common_pb2.pyi | 1 + .../sql/connect/proto/example_plugins_pb2.pyi | 3 + .../sql/connect/proto/expressions_pb2.pyi | 24 ++ .../sql/connect/proto/relations_pb2.py | 258 +++++++++-------- .../sql/connect/proto/relations_pb2.pyi | 143 ++++++++++ .../pyspark/sql/connect/proto/types_pb2.pyi | 26 ++ 13 files changed, 785 insertions(+), 124 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index b9f3c01f1ff7..3402e44c8127 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -66,6 +66,7 @@ message Relation { GroupMap group_map = 31; CoGroupMap co_group_map = 32; WithWatermark with_watermark = 33; + ApplyInPandasWithState apply_in_pandas_with_state = 34; // NA functions NAFill fill_na = 90; @@ -840,6 +841,22 @@ message CoGroupMap { CommonInlineUserDefinedFunction func = 5; } +message ApplyInPandasWithState { + Relation input = 1; + + repeated Expression grouping_expressions = 2; + + CommonInlineUserDefinedFunction func = 3; + + DataType output_schema = 4; + + DataType state_schema = 5; + + string output_mode = 6; + + string timeout_conf = 7; +} + // Collect arbitrary (named) metrics from a dataset. message CollectMetrics { // (Required) The input relation. diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index d8f1615a5967..471237cf820e 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -133,6 +133,8 @@ class SparkConnectPlanner(val session: SparkSession) { transformGroupMap(rel.getGroupMap) case proto.Relation.RelTypeCase.CO_GROUP_MAP => transformCoGroupMap(rel.getCoGroupMap) + case proto.Relation.RelTypeCase.APPLY_IN_PANDAS_WITH_STATE => + transformApplyInPandasWithState(rel.getApplyInPandasWithState) case proto.Relation.RelTypeCase.COLLECT_METRICS => transformCollectMetrics(rel.getCollectMetrics) case proto.Relation.RelTypeCase.PARSE => transformParse(rel.getParse) @@ -583,6 +585,25 @@ class SparkConnectPlanner(val session: SparkSession) { input.flatMapCoGroupsInPandas(other, pythonUdf).logicalPlan } + private def transformApplyInPandasWithState(rel: proto.ApplyInPandasWithState): LogicalPlan = { + val pythonUdf = transformPythonUDF(rel.getFunc) + val cols = + rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) + + val outputSchema = transformDataType(rel.getOutputSchema) + assert(outputSchema.isInstanceOf[StructType]) + + val stateSchema = transformDataType(rel.getStateSchema) + assert(stateSchema.isInstanceOf[StructType]) + + Dataset + .ofRows(session, transformRelation(rel.getInput)) + .groupBy(cols: _*) + .applyInPandasWithState(pythonUdf, outputSchema.asInstanceOf[StructType], + stateSchema.asInstanceOf[StructType], rel.getOutputMode, rel.getTimeoutConf) + .logicalPlan + } + private def transformWithColumnsRenamed(rel: proto.WithColumnsRenamed): LogicalPlan = { Dataset .ofRows(session, transformRelation(rel.getInput)) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 8377caac592d..ce7dff8c19f1 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -35,6 +35,7 @@ from pyspark.rdd import PythonEvalType from pyspark.sql.group import GroupedData as PySparkGroupedData from pyspark.sql.pandas.group_ops import PandasCogroupedOps as PySparkPandasCogroupedOps +from pyspark.sql.types import DataType from pyspark.sql.types import NumericType import pyspark.sql.connect.plan as plan @@ -262,8 +263,272 @@ def applyInPandas( applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ - def applyInPandasWithState(self, *args: Any, **kwargs: Any) -> None: - raise NotImplementedError("applyInPandasWithState() is not implemented.") + +from pyspark.sql.types import NumericType + +import pyspark.sql.connect.plan as plan +from pyspark.sql.connect.column import Column +from pyspark.sql.connect.functions import _invoke_function, col, lit + +if TYPE_CHECKING: + from pyspark.sql.connect._typing import ( + LiteralType, + PandasGroupedMapFunction, + GroupedMapPandasUserDefinedFunction, + PandasCogroupedMapFunction, + ) + from pyspark.sql.connect.dataframe import DataFrame + from pyspark.sql.types import StructType + + +class GroupedData: + def __init__( + self, + df: "DataFrame", + group_type: str, + grouping_cols: Sequence["Column"], + pivot_col: Optional["Column"] = None, + pivot_values: Optional[Sequence["LiteralType"]] = None, + ) -> None: + from pyspark.sql.connect.dataframe import DataFrame + + assert isinstance(df, DataFrame) + self._df = df + + assert isinstance(group_type, str) and group_type in ["groupby", "rollup", "cube", "pivot"] + self._group_type = group_type + + assert isinstance(grouping_cols, list) and all(isinstance(g, Column) for g in grouping_cols) + self._grouping_cols: List[Column] = grouping_cols + + self._pivot_col: Optional["Column"] = None + self._pivot_values: Optional[List[Any]] = None + if group_type == "pivot": + assert pivot_col is not None and isinstance(pivot_col, Column) + assert pivot_values is None or isinstance(pivot_values, list) + self._pivot_col = pivot_col + self._pivot_values = pivot_values + + @overload + def agg(self, *exprs: Column) -> "DataFrame": + ... + + @overload + def agg(self, __exprs: Dict[str, str]) -> "DataFrame": + ... + + def agg(self, *exprs: Union[Column, Dict[str, str]]) -> "DataFrame": + from pyspark.sql.connect.dataframe import DataFrame + + assert exprs, "exprs should not be empty" + if len(exprs) == 1 and isinstance(exprs[0], dict): + # Convert the dict into key value pairs + aggregate_cols = [_invoke_function(exprs[0][k], col(k)) for k in exprs[0]] + else: + # Columns + assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" + aggregate_cols = cast(List[Column], list(exprs)) + + return DataFrame.withPlan( + plan.Aggregate( + child=self._df._plan, + group_type=self._group_type, + grouping_cols=self._grouping_cols, + aggregate_cols=aggregate_cols, + pivot_col=self._pivot_col, + pivot_values=self._pivot_values, + ), + session=self._df._session, + ) + + agg.__doc__ = PySparkGroupedData.agg.__doc__ + + def _numeric_agg(self, function: str, cols: Sequence[str]) -> "DataFrame": + from pyspark.sql.connect.dataframe import DataFrame + + assert isinstance(function, str) and function in ["min", "max", "avg", "sum"] + + assert isinstance(cols, list) and all(isinstance(c, str) for c in cols) + + schema = self._df.schema + + numerical_cols: List[str] = [ + field.name for field in schema.fields if isinstance(field.dataType, NumericType) + ] + + agg_cols: List[str] = [] + + if len(cols) > 0: + invalid_cols = [c for c in cols if c not in numerical_cols] + if len(invalid_cols) > 0: + raise TypeError( + f"{invalid_cols} are not numeric columns. " + f"Numeric aggregation function can only be applied on numeric columns." + ) + agg_cols = cols + else: + # if no column is provided, then all numerical columns are selected + agg_cols = numerical_cols + + return DataFrame.withPlan( + plan.Aggregate( + child=self._df._plan, + group_type=self._group_type, + grouping_cols=self._grouping_cols, + aggregate_cols=[_invoke_function(function, col(c)) for c in agg_cols], + pivot_col=self._pivot_col, + pivot_values=self._pivot_values, + ), + session=self._df._session, + ) + + def min(self, *cols: str) -> "DataFrame": + return self._numeric_agg("min", list(cols)) + + min.__doc__ = PySparkGroupedData.min.__doc__ + + def max(self, *cols: str) -> "DataFrame": + return self._numeric_agg("max", list(cols)) + + max.__doc__ = PySparkGroupedData.max.__doc__ + + def sum(self, *cols: str) -> "DataFrame": + return self._numeric_agg("sum", list(cols)) + + sum.__doc__ = PySparkGroupedData.sum.__doc__ + + def avg(self, *cols: str) -> "DataFrame": + return self._numeric_agg("avg", list(cols)) + + avg.__doc__ = PySparkGroupedData.avg.__doc__ + + mean = avg + + def count(self) -> "DataFrame": + return self.agg(_invoke_function("count", lit(1)).alias("count")) + + count.__doc__ = PySparkGroupedData.count.__doc__ + + def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> "GroupedData": + if self._group_type != "groupby": + if self._group_type == "pivot": + raise Exception("Repeated PIVOT operation is not supported!") + else: + raise Exception(f"PIVOT after {self._group_type.upper()} is not supported!") + + if not isinstance(pivot_col, str): + raise TypeError( + f"pivot_col should be a str, but got {type(pivot_col).__name__} {pivot_col}" + ) + + if values is not None: + if not isinstance(values, list): + raise TypeError( + f"values should be a list, but got {type(values).__name__} {values}" + ) + for v in values: + if not isinstance(v, (bool, float, int, str)): + raise TypeError( + f"value should be a bool, float, int or str, but got {type(v).__name__} {v}" + ) + + return GroupedData( + df=self._df, + group_type="pivot", + grouping_cols=self._grouping_cols, + pivot_col=self._df[pivot_col], + pivot_values=values, + ) + + pivot.__doc__ = PySparkGroupedData.pivot.__doc__ + + def apply(self, udf: "GroupedMapPandasUserDefinedFunction") -> "DataFrame": + # Columns are special because hasattr always return True + if ( + isinstance(udf, Column) + or not hasattr(udf, "func") + or ( + udf.evalType # type: ignore[attr-defined] + != PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF + ) + ): + raise ValueError( + "Invalid udf: the udf argument must be a pandas_udf of type " "GROUPED_MAP." + ) + + warnings.warn( + "It is preferred to use 'applyInPandas' over this " + "API. This API will be deprecated in the future releases. See SPARK-28264 for " + "more details.", + UserWarning, + ) + + return self.applyInPandas(udf.func, schema=udf.returnType) # type: ignore[attr-defined] + + apply.__doc__ = PySparkGroupedData.apply.__doc__ + + def applyInPandas( + self, func: "PandasGroupedMapFunction", schema: Union["StructType", str] + ) -> "DataFrame": + from pyspark.sql.connect.udf import UserDefinedFunction + from pyspark.sql.connect.dataframe import DataFrame + + udf_obj = UserDefinedFunction( + func, + returnType=schema, + evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, + ) + + return DataFrame.withPlan( + plan.GroupMap( + child=self._df._plan, + grouping_cols=self._grouping_cols, + function=udf_obj, + cols=self._df.columns, + ), + session=self._df._session, + ) + + applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ + + def applyInPandasWithState(self, func: "PandasGroupedMapFunction", + output_schema: Union["StructType", str], + state_schema: Union["StructType", str], + output_mode: str, + timeout_conf: str) -> "DataFrame": + from pyspark.sql.connect.udf import UserDefinedFunction + from pyspark.sql.connect.dataframe import DataFrame + from pyspark.sql.connect.types import UnparsedDataType + + udf_obj = UserDefinedFunction( + func, + returnType=output_schema, + evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE, + ) + + output_type: DataType = ( + UnparsedDataType(output_schema) if isinstance(output_schema, str) else output_schema + ) + + state_type: DataType = ( + UnparsedDataType(state_schema) if isinstance(state_schema, str) else state_schema + ) + + return DataFrame.withPlan( + plan.ApplyInPandasWithState( + child=self._df._plan, + grouping_cols=self._grouping_cols, + function=udf_obj, + output_schema=output_type, + state_schema=state_type, + output_mode=output_mode, + timeout_conf=timeout_conf, + cols=self._df.columns, + ), + session=self._df._session, + ) + + applyInPandasWithState.__doc__ = PySparkGroupedData.applyInPandasWithState.__doc__ def cogroup(self, other: "GroupedData") -> "PandasCogroupedOps": return PandasCogroupedOps(self, other) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 9e8770a80da9..e841934490b0 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2045,6 +2045,46 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return plan +class ApplyInPandasWithState(LogicalPlan): + """Logical plan object for a applyInPandasWithState.""" + def __init__( + self, + child: Optional["LogicalPlan"], + grouping_cols: Sequence[Column], + function: "UserDefinedFunction", + output_schema: DataType, + state_schema: DataType, + output_mode: str, + timeout_conf: str, + cols: List[str], + ): + assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) + + super().__init__(child) + self._grouping_cols = grouping_cols + self._func = function._build_common_inline_user_defined_function(*cols) + self._output_schema = output_schema + self._state_schema = state_schema + self._output_mode = output_mode + self._timeout_conf = timeout_conf + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.apply_in_pandas_with_state.input.CopyFrom(self._child.plan(session)) + plan.apply_in_pandas_with_state.grouping_expressions.extend( + [c.to_plan(session) for c in self._grouping_cols] + ) + plan.apply_in_pandas_with_state.func.CopyFrom(self._func.to_plan_udf(session)) + plan.apply_in_pandas_with_state.output_schema \ + .CopyFrom(pyspark_types_to_proto_types(self._output_schema)) + plan.apply_in_pandas_with_state.state_schema \ + .CopyFrom(pyspark_types_to_proto_types(self._state_schema)) + plan.apply_in_pandas_with_state.output_mode = self._output_mode + plan.apply_in_pandas_with_state.timeout_conf = self._timeout_conf + return plan + + class CachedRelation(LogicalPlan): def __init__(self, plan: proto.Relation) -> None: super(CachedRelation, self).__init__(None) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 15e5fb53f140..9ae9a8cd249b 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -55,6 +55,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference @@ -94,6 +95,7 @@ class Plan(google.protobuf.message.Message): global___Plan = Plan +@typing_extensions.final class UserContext(google.protobuf.message.Message): """User Context is used to refer to one particular user session that is executing queries in the backend. @@ -133,11 +135,13 @@ class UserContext(google.protobuf.message.Message): global___UserContext = UserContext +@typing_extensions.final class AnalyzePlanRequest(google.protobuf.message.Message): """Request to perform plan analyze, optionally to explain the plan.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -155,6 +159,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class Explain(google.protobuf.message.Message): """Explains the input plan based on a configurable mode.""" @@ -229,6 +234,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): field_name: typing_extensions.Literal["explain_mode", b"explain_mode", "plan", b"plan"], ) -> None: ... + @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -246,6 +252,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -263,6 +270,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -280,6 +288,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -297,6 +306,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -304,6 +314,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -319,6 +330,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... + @typing_extensions.final class SameSemantics(google.protobuf.message.Message): """Returns `true` when the logical query plans are equal and therefore return same results.""" @@ -351,6 +363,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -368,6 +381,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... + @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -411,6 +425,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_storage_level", b"_storage_level"] ) -> typing_extensions.Literal["storage_level"] | None: ... + @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -443,6 +458,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_blocking", b"_blocking"] ) -> typing_extensions.Literal["blocking"] | None: ... + @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -644,6 +660,7 @@ class AnalyzePlanRequest(google.protobuf.message.Message): global___AnalyzePlanRequest = AnalyzePlanRequest +@typing_extensions.final class AnalyzePlanResponse(google.protobuf.message.Message): """Response to performing analysis of the query. Contains relevant metadata to be able to reason about the performance. @@ -651,6 +668,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -669,6 +687,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["schema", b"schema"] ) -> None: ... + @typing_extensions.final class Explain(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -683,6 +702,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["explain_string", b"explain_string"] ) -> None: ... + @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -697,6 +717,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["tree_string", b"tree_string"] ) -> None: ... + @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -711,6 +732,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_local", b"is_local"] ) -> None: ... + @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -725,6 +747,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_streaming", b"is_streaming"] ) -> None: ... + @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -741,6 +764,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["files", b"files"]) -> None: ... + @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -755,6 +779,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["version", b"version"] ) -> None: ... + @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -773,6 +798,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["parsed", b"parsed"] ) -> None: ... + @typing_extensions.final class SameSemantics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -787,6 +813,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... + @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -801,6 +828,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... + @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -808,6 +836,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -815,6 +844,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... + @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -981,6 +1011,7 @@ class AnalyzePlanResponse(google.protobuf.message.Message): global___AnalyzePlanResponse = AnalyzePlanResponse +@typing_extensions.final class ExecutePlanRequest(google.protobuf.message.Message): """A request to be executed by the service.""" @@ -1054,6 +1085,7 @@ class ExecutePlanRequest(google.protobuf.message.Message): global___ExecutePlanRequest = ExecutePlanRequest +@typing_extensions.final class ExecutePlanResponse(google.protobuf.message.Message): """The response of a query, can be one or more for each request. Responses belonging to the same input query, carry the same `session_id`. @@ -1061,6 +1093,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class SqlCommandResult(google.protobuf.message.Message): """A SQL command returns an opaque Relation that can be directly used as input for the next call. @@ -1083,6 +1116,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... + @typing_extensions.final class ArrowBatch(google.protobuf.message.Message): """Batch results of metrics.""" @@ -1102,12 +1136,15 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "row_count", b"row_count"] ) -> None: ... + @typing_extensions.final class Metrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class MetricObject(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ExecutionMetricsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1167,6 +1204,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class MetricValue(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1207,6 +1245,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["metrics", b"metrics"] ) -> None: ... + @typing_extensions.final class ObservedMetrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1362,6 +1401,7 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse +@typing_extensions.final class KeyValue(google.protobuf.message.Message): """The key-value pair for the config request and response.""" @@ -1394,11 +1434,13 @@ class KeyValue(google.protobuf.message.Message): global___KeyValue = KeyValue +@typing_extensions.final class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Operation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1482,6 +1524,7 @@ class ConfigRequest(google.protobuf.message.Message): "set", "get", "get_with_default", "get_option", "get_all", "unset", "is_modifiable" ] | None: ... + @typing_extensions.final class Set(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1498,6 +1541,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + @typing_extensions.final class Get(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1514,6 +1558,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class GetWithDefault(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1530,6 +1575,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + @typing_extensions.final class GetOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1546,6 +1592,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class GetAll(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1567,6 +1614,7 @@ class ConfigRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] ) -> typing_extensions.Literal["prefix"] | None: ... + @typing_extensions.final class Unset(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1583,6 +1631,7 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... + @typing_extensions.final class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1663,6 +1712,7 @@ class ConfigRequest(google.protobuf.message.Message): global___ConfigRequest = ConfigRequest +@typing_extensions.final class ConfigResponse(google.protobuf.message.Message): """Response to the config request.""" @@ -1705,11 +1755,13 @@ class ConfigResponse(google.protobuf.message.Message): global___ConfigResponse = ConfigResponse +@typing_extensions.final class AddArtifactsRequest(google.protobuf.message.Message): """Request to transfer client-local artifacts.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArtifactChunk(google.protobuf.message.Message): """A chunk of an Artifact.""" @@ -1731,6 +1783,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["crc", b"crc", "data", b"data"] ) -> None: ... + @typing_extensions.final class SingleChunkArtifact(google.protobuf.message.Message): """An artifact that is contained in a single `ArtifactChunk`. Generally, this message represents tiny artifacts such as REPL-generated class files. @@ -1764,6 +1817,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "name", b"name"] ) -> None: ... + @typing_extensions.final class Batch(google.protobuf.message.Message): """A number of `SingleChunkArtifact` batched into a single RPC.""" @@ -1786,6 +1840,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["artifacts", b"artifacts"] ) -> None: ... + @typing_extensions.final class BeginChunkedArtifact(google.protobuf.message.Message): """Signals the beginning/start of a chunked artifact. A large artifact is transferred through a payload of `BeginChunkedArtifact` followed by a @@ -1931,6 +1986,7 @@ class AddArtifactsRequest(google.protobuf.message.Message): global___AddArtifactsRequest = AddArtifactsRequest +@typing_extensions.final class AddArtifactsResponse(google.protobuf.message.Message): """Response to adding an artifact. Contains relevant metadata to verify successful transfer of artifact(s). @@ -1938,6 +1994,7 @@ class AddArtifactsResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArtifactSummary(google.protobuf.message.Message): """Metadata of an artifact.""" diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.pyi b/python/pyspark/sql/connect/proto/catalog_pb2.pyi index 6e0ee5f5c311..b550cf191e1d 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.pyi +++ b/python/pyspark/sql/connect/proto/catalog_pb2.pyi @@ -50,6 +50,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Catalog(google.protobuf.message.Message): """Catalog messages are marked as unstable.""" @@ -314,6 +315,7 @@ class Catalog(google.protobuf.message.Message): global___Catalog = Catalog +@typing_extensions.final class CurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.currentDatabase`""" @@ -325,6 +327,7 @@ class CurrentDatabase(google.protobuf.message.Message): global___CurrentDatabase = CurrentDatabase +@typing_extensions.final class SetCurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.setCurrentDatabase`""" @@ -342,6 +345,7 @@ class SetCurrentDatabase(google.protobuf.message.Message): global___SetCurrentDatabase = SetCurrentDatabase +@typing_extensions.final class ListDatabases(google.protobuf.message.Message): """See `spark.catalog.listDatabases`""" @@ -353,6 +357,7 @@ class ListDatabases(google.protobuf.message.Message): global___ListDatabases = ListDatabases +@typing_extensions.final class ListTables(google.protobuf.message.Message): """See `spark.catalog.listTables`""" @@ -378,6 +383,7 @@ class ListTables(google.protobuf.message.Message): global___ListTables = ListTables +@typing_extensions.final class ListFunctions(google.protobuf.message.Message): """See `spark.catalog.listFunctions`""" @@ -403,6 +409,7 @@ class ListFunctions(google.protobuf.message.Message): global___ListFunctions = ListFunctions +@typing_extensions.final class ListColumns(google.protobuf.message.Message): """See `spark.catalog.listColumns`""" @@ -435,6 +442,7 @@ class ListColumns(google.protobuf.message.Message): global___ListColumns = ListColumns +@typing_extensions.final class GetDatabase(google.protobuf.message.Message): """See `spark.catalog.getDatabase`""" @@ -452,6 +460,7 @@ class GetDatabase(google.protobuf.message.Message): global___GetDatabase = GetDatabase +@typing_extensions.final class GetTable(google.protobuf.message.Message): """See `spark.catalog.getTable`""" @@ -484,6 +493,7 @@ class GetTable(google.protobuf.message.Message): global___GetTable = GetTable +@typing_extensions.final class GetFunction(google.protobuf.message.Message): """See `spark.catalog.getFunction`""" @@ -516,6 +526,7 @@ class GetFunction(google.protobuf.message.Message): global___GetFunction = GetFunction +@typing_extensions.final class DatabaseExists(google.protobuf.message.Message): """See `spark.catalog.databaseExists`""" @@ -533,6 +544,7 @@ class DatabaseExists(google.protobuf.message.Message): global___DatabaseExists = DatabaseExists +@typing_extensions.final class TableExists(google.protobuf.message.Message): """See `spark.catalog.tableExists`""" @@ -565,6 +577,7 @@ class TableExists(google.protobuf.message.Message): global___TableExists = TableExists +@typing_extensions.final class FunctionExists(google.protobuf.message.Message): """See `spark.catalog.functionExists`""" @@ -597,11 +610,13 @@ class FunctionExists(google.protobuf.message.Message): global___FunctionExists = FunctionExists +@typing_extensions.final class CreateExternalTable(google.protobuf.message.Message): """See `spark.catalog.createExternalTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -700,11 +715,13 @@ class CreateExternalTable(google.protobuf.message.Message): global___CreateExternalTable = CreateExternalTable +@typing_extensions.final class CreateTable(google.protobuf.message.Message): """See `spark.catalog.createTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -819,6 +836,7 @@ class CreateTable(google.protobuf.message.Message): global___CreateTable = CreateTable +@typing_extensions.final class DropTempView(google.protobuf.message.Message): """See `spark.catalog.dropTempView`""" @@ -838,6 +856,7 @@ class DropTempView(google.protobuf.message.Message): global___DropTempView = DropTempView +@typing_extensions.final class DropGlobalTempView(google.protobuf.message.Message): """See `spark.catalog.dropGlobalTempView`""" @@ -857,6 +876,7 @@ class DropGlobalTempView(google.protobuf.message.Message): global___DropGlobalTempView = DropGlobalTempView +@typing_extensions.final class RecoverPartitions(google.protobuf.message.Message): """See `spark.catalog.recoverPartitions`""" @@ -876,6 +896,7 @@ class RecoverPartitions(google.protobuf.message.Message): global___RecoverPartitions = RecoverPartitions +@typing_extensions.final class IsCached(google.protobuf.message.Message): """See `spark.catalog.isCached`""" @@ -895,6 +916,7 @@ class IsCached(google.protobuf.message.Message): global___IsCached = IsCached +@typing_extensions.final class CacheTable(google.protobuf.message.Message): """See `spark.catalog.cacheTable`""" @@ -936,6 +958,7 @@ class CacheTable(google.protobuf.message.Message): global___CacheTable = CacheTable +@typing_extensions.final class UncacheTable(google.protobuf.message.Message): """See `spark.catalog.uncacheTable`""" @@ -955,6 +978,7 @@ class UncacheTable(google.protobuf.message.Message): global___UncacheTable = UncacheTable +@typing_extensions.final class ClearCache(google.protobuf.message.Message): """See `spark.catalog.clearCache`""" @@ -966,6 +990,7 @@ class ClearCache(google.protobuf.message.Message): global___ClearCache = ClearCache +@typing_extensions.final class RefreshTable(google.protobuf.message.Message): """See `spark.catalog.refreshTable`""" @@ -985,6 +1010,7 @@ class RefreshTable(google.protobuf.message.Message): global___RefreshTable = RefreshTable +@typing_extensions.final class RefreshByPath(google.protobuf.message.Message): """See `spark.catalog.refreshByPath`""" @@ -1002,6 +1028,7 @@ class RefreshByPath(google.protobuf.message.Message): global___RefreshByPath = RefreshByPath +@typing_extensions.final class CurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.currentCatalog`""" @@ -1013,6 +1040,7 @@ class CurrentCatalog(google.protobuf.message.Message): global___CurrentCatalog = CurrentCatalog +@typing_extensions.final class SetCurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.setCurrentCatalog`""" @@ -1032,6 +1060,7 @@ class SetCurrentCatalog(google.protobuf.message.Message): global___SetCurrentCatalog = SetCurrentCatalog +@typing_extensions.final class ListCatalogs(google.protobuf.message.Message): """See `spark.catalog.listCatalogs`""" diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 419a2bd840e3..5691b16fea31 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -53,6 +53,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Command(google.protobuf.message.Message): """A [[Command]] is an operation that is executed by the server that does not directly consume or produce a relational result. @@ -172,6 +173,7 @@ class Command(google.protobuf.message.Message): global___Command = Command +@typing_extensions.final class SqlCommand(google.protobuf.message.Message): """A SQL Command is used to trigger the eager evaluation of SQL commands in Spark. @@ -183,6 +185,7 @@ class SqlCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -230,6 +233,7 @@ class SqlCommand(google.protobuf.message.Message): global___SqlCommand = SqlCommand +@typing_extensions.final class CreateDataFrameViewCommand(google.protobuf.message.Message): """A command that can create DataFrame global temp view or local temp view.""" @@ -272,6 +276,7 @@ class CreateDataFrameViewCommand(google.protobuf.message.Message): global___CreateDataFrameViewCommand = CreateDataFrameViewCommand +@typing_extensions.final class WriteOperation(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -301,6 +306,7 @@ class WriteOperation(google.protobuf.message.Message): SAVE_MODE_ERROR_IF_EXISTS: WriteOperation.SaveMode.ValueType # 3 SAVE_MODE_IGNORE: WriteOperation.SaveMode.ValueType # 4 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -318,6 +324,7 @@ class WriteOperation(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class SaveTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -360,6 +367,7 @@ class WriteOperation(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class BucketBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -490,6 +498,7 @@ class WriteOperation(google.protobuf.message.Message): global___WriteOperation = WriteOperation +@typing_extensions.final class WriteOperationV2(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -523,6 +532,7 @@ class WriteOperationV2(google.protobuf.message.Message): MODE_REPLACE: WriteOperationV2.Mode.ValueType # 5 MODE_CREATE_OR_REPLACE: WriteOperationV2.Mode.ValueType # 6 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -540,6 +550,7 @@ class WriteOperationV2(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... + @typing_extensions.final class TablePropertiesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -653,6 +664,7 @@ class WriteOperationV2(google.protobuf.message.Message): global___WriteOperationV2 = WriteOperationV2 +@typing_extensions.final class WriteStreamOperationStart(google.protobuf.message.Message): """Starts write stream operation as streaming query. Query ID and Run ID of the streaming query are returned. @@ -660,6 +672,7 @@ class WriteStreamOperationStart(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -797,6 +810,7 @@ class WriteStreamOperationStart(google.protobuf.message.Message): global___WriteStreamOperationStart = WriteStreamOperationStart +@typing_extensions.final class WriteStreamOperationStartResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -822,6 +836,7 @@ class WriteStreamOperationStartResult(google.protobuf.message.Message): global___WriteStreamOperationStartResult = WriteStreamOperationStartResult +@typing_extensions.final class StreamingQueryInstanceId(google.protobuf.message.Message): """A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that persists across the streaming runs and `run_id` that changes between each run of the @@ -854,11 +869,13 @@ class StreamingQueryInstanceId(google.protobuf.message.Message): global___StreamingQueryInstanceId = StreamingQueryInstanceId +@typing_extensions.final class StreamingQueryCommand(google.protobuf.message.Message): """Commands for a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ExplainCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -960,11 +977,13 @@ class StreamingQueryCommand(google.protobuf.message.Message): global___StreamingQueryCommand = StreamingQueryCommand +@typing_extensions.final class StreamingQueryCommandResult(google.protobuf.message.Message): """Response for commands on a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class StatusResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -999,6 +1018,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class RecentProgressResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1018,6 +1038,7 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): field_name: typing_extensions.Literal["recent_progress_json", b"recent_progress_json"], ) -> None: ... + @typing_extensions.final class ExplainResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/common_pb2.pyi b/python/pyspark/sql/connect/proto/common_pb2.pyi index bb7bdeddbfd6..03bf86281946 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/common_pb2.pyi @@ -47,6 +47,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class StorageLevel(google.protobuf.message.Message): """StorageLevel for persisting Datasets/Tables.""" diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi index 1be966ff1e40..4a068a7980b3 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi @@ -47,6 +47,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class ExamplePluginRelation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -71,6 +72,7 @@ class ExamplePluginRelation(google.protobuf.message.Message): global___ExamplePluginRelation = ExamplePluginRelation +@typing_extensions.final class ExamplePluginExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -95,6 +97,7 @@ class ExamplePluginExpression(google.protobuf.message.Message): global___ExamplePluginExpression = ExamplePluginExpression +@typing_extensions.final class ExamplePluginCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 16f84694d2fd..fb2f684546aa 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -51,6 +51,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Expression(google.protobuf.message.Message): """Expression used to refer to fields, functions and similar. This can be used everywhere expressions in SQL appear. @@ -58,11 +59,13 @@ class Expression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Window(google.protobuf.message.Message): """Expression for the OVER clause or WINDOW clause.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class WindowFrame(google.protobuf.message.Message): """The window frame""" @@ -96,6 +99,7 @@ class Expression(google.protobuf.message.Message): All rows having the same 'ORDER BY' ordering are considered as peers. """ + @typing_extensions.final class FrameBoundary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -232,6 +236,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class SortOrder(google.protobuf.message.Message): """SortOrder is used to specify the data ordering, it is normally used in Sort and Window. It is an unevaluable expression and cannot be evaluated, so can not be used in Projection. @@ -306,6 +311,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Cast(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -356,9 +362,11 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["cast_to_type", b"cast_to_type"] ) -> typing_extensions.Literal["type", "type_str"] | None: ... + @typing_extensions.final class Literal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -417,6 +425,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... + @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -440,6 +449,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -643,6 +653,7 @@ class Expression(google.protobuf.message.Message): "array", ] | None: ... + @typing_extensions.final class UnresolvedAttribute(google.protobuf.message.Message): """An unresolved attribute that is not explicitly bound to a specific column, but the column is resolved during analysis by name. @@ -683,6 +694,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... + @typing_extensions.final class UnresolvedFunction(google.protobuf.message.Message): """An unresolved function is not explicitly bound to one explicit function, but the function is resolved during analysis following Sparks name resolution rules. @@ -733,6 +745,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class ExpressionString(google.protobuf.message.Message): """Expression as string.""" @@ -750,6 +763,7 @@ class Expression(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["expression", b"expression"] ) -> None: ... + @typing_extensions.final class UnresolvedStar(google.protobuf.message.Message): """UnresolvedStar is used to expand all the fields of a relation or struct.""" @@ -783,6 +797,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_unparsed_target", b"_unparsed_target"] ) -> typing_extensions.Literal["unparsed_target"] | None: ... + @typing_extensions.final class UnresolvedRegex(google.protobuf.message.Message): """Represents all of the input attributes to a given relational operator, for example in "SELECT `(id)?+.+` FROM ...". @@ -816,6 +831,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... + @typing_extensions.final class UnresolvedExtractValue(google.protobuf.message.Message): """Extracts a value or values from an Expression""" @@ -848,6 +864,7 @@ class Expression(google.protobuf.message.Message): field_name: typing_extensions.Literal["child", b"child", "extraction", b"extraction"], ) -> None: ... + @typing_extensions.final class UpdateFields(google.protobuf.message.Message): """Add, replace or drop a field of `StructType` expression by name.""" @@ -892,6 +909,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Alias(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -934,6 +952,7 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... + @typing_extensions.final class LambdaFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -970,6 +989,7 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class UnresolvedNamedLambdaVariable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1163,6 +1183,7 @@ class Expression(google.protobuf.message.Message): global___Expression = Expression +@typing_extensions.final class CommonInlineUserDefinedFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1235,6 +1256,7 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedFunction = CommonInlineUserDefinedFunction +@typing_extensions.final class PythonUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1278,6 +1300,7 @@ class PythonUDF(google.protobuf.message.Message): global___PythonUDF = PythonUDF +@typing_extensions.final class ScalarScalaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1327,6 +1350,7 @@ class ScalarScalaUDF(google.protobuf.message.Message): global___ScalarScalaUDF = ScalarScalaUDF +@typing_extensions.final class JavaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index c37af023846a..97afa9f503e8 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -36,7 +36,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"\xf6\x14\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\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\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"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xa9\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\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"\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"\xd7\x03\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"\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"\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"\xc6\x04\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\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"\x81\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"\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"\xab\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\x42\x16\n\x14_all_columns_as_keys"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"\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"\\\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"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\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"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"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"\xb5\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\x42\r\n\x0b_is_barrier"\xcb\x01\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"\xe0\x02\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"\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_schemaB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\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"\xdb\x15\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\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\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"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xa9\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\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"\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"\xd7\x03\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"\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"\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"\xc6\x04\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\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"\x81\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"\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"\xab\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\x42\x16\n\x14_all_columns_as_keys"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"\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"\\\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"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\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"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"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"\xb5\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\x42\r\n\x0b_is_barrier"\xcb\x01\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"\xe0\x02\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"\x97\x03\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(\x0b\x32\x17.spark.connect.DataTypeR\x0coutputSchema\x12:\n\x0cstate_schema\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\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_schemaB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -96,6 +96,7 @@ _MAPPARTITIONS = DESCRIPTOR.message_types_by_name["MapPartitions"] _GROUPMAP = DESCRIPTOR.message_types_by_name["GroupMap"] _COGROUPMAP = DESCRIPTOR.message_types_by_name["CoGroupMap"] +_APPLYINPANDASWITHSTATE = DESCRIPTOR.message_types_by_name["ApplyInPandasWithState"] _COLLECTMETRICS = DESCRIPTOR.message_types_by_name["CollectMetrics"] _PARSE = DESCRIPTOR.message_types_by_name["Parse"] _PARSE_OPTIONSENTRY = _PARSE.nested_types_by_name["OptionsEntry"] @@ -687,6 +688,17 @@ ) _sym_db.RegisterMessage(CoGroupMap) +ApplyInPandasWithState = _reflection.GeneratedProtocolMessageType( + "ApplyInPandasWithState", + (_message.Message,), + { + "DESCRIPTOR": _APPLYINPANDASWITHSTATE, + "__module__": "spark.connect.relations_pb2" + # @@protoc_insertion_point(class_scope:spark.connect.ApplyInPandasWithState) + }, +) +_sym_db.RegisterMessage(ApplyInPandasWithState) + CollectMetrics = _reflection.GeneratedProtocolMessageType( "CollectMetrics", (_message.Message,), @@ -734,125 +746,127 @@ _PARSE_OPTIONSENTRY._options = None _PARSE_OPTIONSENTRY._serialized_options = b"8\001" _RELATION._serialized_start = 165 - _RELATION._serialized_end = 2843 - _UNKNOWN._serialized_start = 2845 - _UNKNOWN._serialized_end = 2854 - _RELATIONCOMMON._serialized_start = 2856 - _RELATIONCOMMON._serialized_end = 2947 - _SQL._serialized_start = 2950 - _SQL._serialized_end = 3119 - _SQL_ARGSENTRY._serialized_start = 3029 - _SQL_ARGSENTRY._serialized_end = 3119 - _READ._serialized_start = 3122 - _READ._serialized_end = 3785 - _READ_NAMEDTABLE._serialized_start = 3300 - _READ_NAMEDTABLE._serialized_end = 3492 - _READ_NAMEDTABLE_OPTIONSENTRY._serialized_start = 3434 - _READ_NAMEDTABLE_OPTIONSENTRY._serialized_end = 3492 - _READ_DATASOURCE._serialized_start = 3495 - _READ_DATASOURCE._serialized_end = 3772 - _READ_DATASOURCE_OPTIONSENTRY._serialized_start = 3434 - _READ_DATASOURCE_OPTIONSENTRY._serialized_end = 3492 - _PROJECT._serialized_start = 3787 - _PROJECT._serialized_end = 3904 - _FILTER._serialized_start = 3906 - _FILTER._serialized_end = 4018 - _JOIN._serialized_start = 4021 - _JOIN._serialized_end = 4492 - _JOIN_JOINTYPE._serialized_start = 4284 - _JOIN_JOINTYPE._serialized_end = 4492 - _SETOPERATION._serialized_start = 4495 - _SETOPERATION._serialized_end = 4974 - _SETOPERATION_SETOPTYPE._serialized_start = 4811 - _SETOPERATION_SETOPTYPE._serialized_end = 4925 - _LIMIT._serialized_start = 4976 - _LIMIT._serialized_end = 5052 - _OFFSET._serialized_start = 5054 - _OFFSET._serialized_end = 5133 - _TAIL._serialized_start = 5135 - _TAIL._serialized_end = 5210 - _AGGREGATE._serialized_start = 5213 - _AGGREGATE._serialized_end = 5795 - _AGGREGATE_PIVOT._serialized_start = 5552 - _AGGREGATE_PIVOT._serialized_end = 5663 - _AGGREGATE_GROUPTYPE._serialized_start = 5666 - _AGGREGATE_GROUPTYPE._serialized_end = 5795 - _SORT._serialized_start = 5798 - _SORT._serialized_end = 5958 - _DROP._serialized_start = 5961 - _DROP._serialized_end = 6102 - _DEDUPLICATE._serialized_start = 6105 - _DEDUPLICATE._serialized_end = 6276 - _LOCALRELATION._serialized_start = 6278 - _LOCALRELATION._serialized_end = 6367 - _SAMPLE._serialized_start = 6370 - _SAMPLE._serialized_end = 6643 - _RANGE._serialized_start = 6646 - _RANGE._serialized_end = 6791 - _SUBQUERYALIAS._serialized_start = 6793 - _SUBQUERYALIAS._serialized_end = 6907 - _REPARTITION._serialized_start = 6910 - _REPARTITION._serialized_end = 7052 - _SHOWSTRING._serialized_start = 7055 - _SHOWSTRING._serialized_end = 7197 - _STATSUMMARY._serialized_start = 7199 - _STATSUMMARY._serialized_end = 7291 - _STATDESCRIBE._serialized_start = 7293 - _STATDESCRIBE._serialized_end = 7374 - _STATCROSSTAB._serialized_start = 7376 - _STATCROSSTAB._serialized_end = 7477 - _STATCOV._serialized_start = 7479 - _STATCOV._serialized_end = 7575 - _STATCORR._serialized_start = 7578 - _STATCORR._serialized_end = 7715 - _STATAPPROXQUANTILE._serialized_start = 7718 - _STATAPPROXQUANTILE._serialized_end = 7882 - _STATFREQITEMS._serialized_start = 7884 - _STATFREQITEMS._serialized_end = 8009 - _STATSAMPLEBY._serialized_start = 8012 - _STATSAMPLEBY._serialized_end = 8321 - _STATSAMPLEBY_FRACTION._serialized_start = 8213 - _STATSAMPLEBY_FRACTION._serialized_end = 8312 - _NAFILL._serialized_start = 8324 - _NAFILL._serialized_end = 8458 - _NADROP._serialized_start = 8461 - _NADROP._serialized_end = 8595 - _NAREPLACE._serialized_start = 8598 - _NAREPLACE._serialized_end = 8894 - _NAREPLACE_REPLACEMENT._serialized_start = 8753 - _NAREPLACE_REPLACEMENT._serialized_end = 8894 - _TODF._serialized_start = 8896 - _TODF._serialized_end = 8984 - _WITHCOLUMNSRENAMED._serialized_start = 8987 - _WITHCOLUMNSRENAMED._serialized_end = 9226 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 9159 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 9226 - _WITHCOLUMNS._serialized_start = 9228 - _WITHCOLUMNS._serialized_end = 9347 - _WITHWATERMARK._serialized_start = 9350 - _WITHWATERMARK._serialized_end = 9484 - _HINT._serialized_start = 9487 - _HINT._serialized_end = 9619 - _UNPIVOT._serialized_start = 9622 - _UNPIVOT._serialized_end = 9949 - _UNPIVOT_VALUES._serialized_start = 9879 - _UNPIVOT_VALUES._serialized_end = 9938 - _TOSCHEMA._serialized_start = 9951 - _TOSCHEMA._serialized_end = 10057 - _REPARTITIONBYEXPRESSION._serialized_start = 10060 - _REPARTITIONBYEXPRESSION._serialized_end = 10263 - _MAPPARTITIONS._serialized_start = 10266 - _MAPPARTITIONS._serialized_end = 10447 - _GROUPMAP._serialized_start = 10450 - _GROUPMAP._serialized_end = 10653 - _COGROUPMAP._serialized_start = 10656 - _COGROUPMAP._serialized_end = 11008 - _COLLECTMETRICS._serialized_start = 11011 - _COLLECTMETRICS._serialized_end = 11147 - _PARSE._serialized_start = 11150 - _PARSE._serialized_end = 11538 - _PARSE_OPTIONSENTRY._serialized_start = 3434 - _PARSE_OPTIONSENTRY._serialized_end = 3492 - _PARSE_PARSEFORMAT._serialized_start = 11439 - _PARSE_PARSEFORMAT._serialized_end = 11527 + _RELATION._serialized_end = 2944 + _UNKNOWN._serialized_start = 2946 + _UNKNOWN._serialized_end = 2955 + _RELATIONCOMMON._serialized_start = 2957 + _RELATIONCOMMON._serialized_end = 3048 + _SQL._serialized_start = 3051 + _SQL._serialized_end = 3220 + _SQL_ARGSENTRY._serialized_start = 3130 + _SQL_ARGSENTRY._serialized_end = 3220 + _READ._serialized_start = 3223 + _READ._serialized_end = 3886 + _READ_NAMEDTABLE._serialized_start = 3401 + _READ_NAMEDTABLE._serialized_end = 3593 + _READ_NAMEDTABLE_OPTIONSENTRY._serialized_start = 3535 + _READ_NAMEDTABLE_OPTIONSENTRY._serialized_end = 3593 + _READ_DATASOURCE._serialized_start = 3596 + _READ_DATASOURCE._serialized_end = 3873 + _READ_DATASOURCE_OPTIONSENTRY._serialized_start = 3535 + _READ_DATASOURCE_OPTIONSENTRY._serialized_end = 3593 + _PROJECT._serialized_start = 3888 + _PROJECT._serialized_end = 4005 + _FILTER._serialized_start = 4007 + _FILTER._serialized_end = 4119 + _JOIN._serialized_start = 4122 + _JOIN._serialized_end = 4593 + _JOIN_JOINTYPE._serialized_start = 4385 + _JOIN_JOINTYPE._serialized_end = 4593 + _SETOPERATION._serialized_start = 4596 + _SETOPERATION._serialized_end = 5075 + _SETOPERATION_SETOPTYPE._serialized_start = 4912 + _SETOPERATION_SETOPTYPE._serialized_end = 5026 + _LIMIT._serialized_start = 5077 + _LIMIT._serialized_end = 5153 + _OFFSET._serialized_start = 5155 + _OFFSET._serialized_end = 5234 + _TAIL._serialized_start = 5236 + _TAIL._serialized_end = 5311 + _AGGREGATE._serialized_start = 5314 + _AGGREGATE._serialized_end = 5896 + _AGGREGATE_PIVOT._serialized_start = 5653 + _AGGREGATE_PIVOT._serialized_end = 5764 + _AGGREGATE_GROUPTYPE._serialized_start = 5767 + _AGGREGATE_GROUPTYPE._serialized_end = 5896 + _SORT._serialized_start = 5899 + _SORT._serialized_end = 6059 + _DROP._serialized_start = 6062 + _DROP._serialized_end = 6203 + _DEDUPLICATE._serialized_start = 6206 + _DEDUPLICATE._serialized_end = 6377 + _LOCALRELATION._serialized_start = 6379 + _LOCALRELATION._serialized_end = 6468 + _SAMPLE._serialized_start = 6471 + _SAMPLE._serialized_end = 6744 + _RANGE._serialized_start = 6747 + _RANGE._serialized_end = 6892 + _SUBQUERYALIAS._serialized_start = 6894 + _SUBQUERYALIAS._serialized_end = 7008 + _REPARTITION._serialized_start = 7011 + _REPARTITION._serialized_end = 7153 + _SHOWSTRING._serialized_start = 7156 + _SHOWSTRING._serialized_end = 7298 + _STATSUMMARY._serialized_start = 7300 + _STATSUMMARY._serialized_end = 7392 + _STATDESCRIBE._serialized_start = 7394 + _STATDESCRIBE._serialized_end = 7475 + _STATCROSSTAB._serialized_start = 7477 + _STATCROSSTAB._serialized_end = 7578 + _STATCOV._serialized_start = 7580 + _STATCOV._serialized_end = 7676 + _STATCORR._serialized_start = 7679 + _STATCORR._serialized_end = 7816 + _STATAPPROXQUANTILE._serialized_start = 7819 + _STATAPPROXQUANTILE._serialized_end = 7983 + _STATFREQITEMS._serialized_start = 7985 + _STATFREQITEMS._serialized_end = 8110 + _STATSAMPLEBY._serialized_start = 8113 + _STATSAMPLEBY._serialized_end = 8422 + _STATSAMPLEBY_FRACTION._serialized_start = 8314 + _STATSAMPLEBY_FRACTION._serialized_end = 8413 + _NAFILL._serialized_start = 8425 + _NAFILL._serialized_end = 8559 + _NADROP._serialized_start = 8562 + _NADROP._serialized_end = 8696 + _NAREPLACE._serialized_start = 8699 + _NAREPLACE._serialized_end = 8995 + _NAREPLACE_REPLACEMENT._serialized_start = 8854 + _NAREPLACE_REPLACEMENT._serialized_end = 8995 + _TODF._serialized_start = 8997 + _TODF._serialized_end = 9085 + _WITHCOLUMNSRENAMED._serialized_start = 9088 + _WITHCOLUMNSRENAMED._serialized_end = 9327 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 9260 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 9327 + _WITHCOLUMNS._serialized_start = 9329 + _WITHCOLUMNS._serialized_end = 9448 + _WITHWATERMARK._serialized_start = 9451 + _WITHWATERMARK._serialized_end = 9585 + _HINT._serialized_start = 9588 + _HINT._serialized_end = 9720 + _UNPIVOT._serialized_start = 9723 + _UNPIVOT._serialized_end = 10050 + _UNPIVOT_VALUES._serialized_start = 9980 + _UNPIVOT_VALUES._serialized_end = 10039 + _TOSCHEMA._serialized_start = 10052 + _TOSCHEMA._serialized_end = 10158 + _REPARTITIONBYEXPRESSION._serialized_start = 10161 + _REPARTITIONBYEXPRESSION._serialized_end = 10364 + _MAPPARTITIONS._serialized_start = 10367 + _MAPPARTITIONS._serialized_end = 10548 + _GROUPMAP._serialized_start = 10551 + _GROUPMAP._serialized_end = 10754 + _COGROUPMAP._serialized_start = 10757 + _COGROUPMAP._serialized_end = 11109 + _APPLYINPANDASWITHSTATE._serialized_start = 11112 + _APPLYINPANDASWITHSTATE._serialized_end = 11519 + _COLLECTMETRICS._serialized_start = 11522 + _COLLECTMETRICS._serialized_end = 11658 + _PARSE._serialized_start = 11661 + _PARSE._serialized_end = 12049 + _PARSE_OPTIONSENTRY._serialized_start = 3535 + _PARSE_OPTIONSENTRY._serialized_end = 3593 + _PARSE_PARSEFORMAT._serialized_start = 11950 + _PARSE_PARSEFORMAT._serialized_end = 12038 # @@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 664206c9c930..d91375ebb78e 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -53,6 +53,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class Relation(google.protobuf.message.Message): """The main [[Relation]] type. Fundamentally, a relation is a typed container that has exactly one explicit relation type set. @@ -95,6 +96,7 @@ class Relation(google.protobuf.message.Message): GROUP_MAP_FIELD_NUMBER: builtins.int CO_GROUP_MAP_FIELD_NUMBER: builtins.int WITH_WATERMARK_FIELD_NUMBER: builtins.int + APPLY_IN_PANDAS_WITH_STATE_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -176,6 +178,8 @@ class Relation(google.protobuf.message.Message): @property def with_watermark(self) -> global___WithWatermark: ... @property + def apply_in_pandas_with_state(self) -> global___ApplyInPandasWithState: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -245,6 +249,7 @@ class Relation(google.protobuf.message.Message): group_map: global___GroupMap | None = ..., co_group_map: global___CoGroupMap | None = ..., with_watermark: global___WithWatermark | None = ..., + apply_in_pandas_with_state: global___ApplyInPandasWithState | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -265,6 +270,8 @@ class Relation(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "aggregate", b"aggregate", + "apply_in_pandas_with_state", + b"apply_in_pandas_with_state", "approx_quantile", b"approx_quantile", "catalog", @@ -366,6 +373,8 @@ class Relation(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "aggregate", b"aggregate", + "apply_in_pandas_with_state", + b"apply_in_pandas_with_state", "approx_quantile", b"approx_quantile", "catalog", @@ -497,6 +506,7 @@ class Relation(google.protobuf.message.Message): "group_map", "co_group_map", "with_watermark", + "apply_in_pandas_with_state", "fill_na", "drop_na", "replace", @@ -515,6 +525,7 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation +@typing_extensions.final class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" @@ -526,6 +537,7 @@ class Unknown(google.protobuf.message.Message): global___Unknown = Unknown +@typing_extensions.final class RelationCommon(google.protobuf.message.Message): """Common metadata of all relations.""" @@ -558,11 +570,13 @@ class RelationCommon(google.protobuf.message.Message): global___RelationCommon = RelationCommon +@typing_extensions.final class SQL(google.protobuf.message.Message): """Relation that uses a SQL query to generate the output.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -610,6 +624,7 @@ class SQL(google.protobuf.message.Message): global___SQL = SQL +@typing_extensions.final class Read(google.protobuf.message.Message): """Relation that reads from a file / table or other data source. Does not have additional inputs. @@ -617,9 +632,11 @@ class Read(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class NamedTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -659,9 +676,11 @@ class Read(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class DataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -806,6 +825,7 @@ class Read(google.protobuf.message.Message): global___Read = Read +@typing_extensions.final class Project(google.protobuf.message.Message): """Projection of a bag of expressions for a given input relation. @@ -847,6 +867,7 @@ class Project(google.protobuf.message.Message): global___Project = Project +@typing_extensions.final class Filter(google.protobuf.message.Message): """Relation that applies a boolean expression `condition` on each row of `input` to produce the output result. @@ -877,6 +898,7 @@ class Filter(google.protobuf.message.Message): global___Filter = Filter +@typing_extensions.final class Join(google.protobuf.message.Message): """Relation of type [[Join]]. @@ -975,6 +997,7 @@ class Join(google.protobuf.message.Message): global___Join = Join +@typing_extensions.final class SetOperation(google.protobuf.message.Message): """Relation of type [[SetOperation]]""" @@ -1102,6 +1125,7 @@ class SetOperation(google.protobuf.message.Message): global___SetOperation = SetOperation +@typing_extensions.final class Limit(google.protobuf.message.Message): """Relation of type [[Limit]] that is used to `limit` rows from the input relation.""" @@ -1129,6 +1153,7 @@ class Limit(google.protobuf.message.Message): global___Limit = Limit +@typing_extensions.final class Offset(google.protobuf.message.Message): """Relation of type [[Offset]] that is used to read rows staring from the `offset` on the input relation. @@ -1158,6 +1183,7 @@ class Offset(google.protobuf.message.Message): global___Offset = Offset +@typing_extensions.final class Tail(google.protobuf.message.Message): """Relation of type [[Tail]] that is used to fetch `limit` rows from the last of the input relation.""" @@ -1185,6 +1211,7 @@ class Tail(google.protobuf.message.Message): global___Tail = Tail +@typing_extensions.final class Aggregate(google.protobuf.message.Message): """Relation of type [[Aggregate]].""" @@ -1212,6 +1239,7 @@ class Aggregate(google.protobuf.message.Message): GROUP_TYPE_CUBE: Aggregate.GroupType.ValueType # 3 GROUP_TYPE_PIVOT: Aggregate.GroupType.ValueType # 4 + @typing_extensions.final class Pivot(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1310,6 +1338,7 @@ class Aggregate(google.protobuf.message.Message): global___Aggregate = Aggregate +@typing_extensions.final class Sort(google.protobuf.message.Message): """Relation of type [[Sort]].""" @@ -1365,6 +1394,7 @@ class Sort(google.protobuf.message.Message): global___Sort = Sort +@typing_extensions.final class Drop(google.protobuf.message.Message): """Drop specified columns.""" @@ -1408,6 +1438,7 @@ class Drop(google.protobuf.message.Message): global___Drop = Drop +@typing_extensions.final class Deduplicate(google.protobuf.message.Message): """Relation of type [[Deduplicate]] which have duplicate rows removed, could consider either only the subset of columns or all the columns. @@ -1472,6 +1503,7 @@ class Deduplicate(google.protobuf.message.Message): global___Deduplicate = Deduplicate +@typing_extensions.final class LocalRelation(google.protobuf.message.Message): """A relation that does not need to be qualified by name.""" @@ -1519,6 +1551,7 @@ class LocalRelation(google.protobuf.message.Message): global___LocalRelation = LocalRelation +@typing_extensions.final class Sample(google.protobuf.message.Message): """Relation of type [[Sample]] that samples a fraction of the dataset.""" @@ -1603,6 +1636,7 @@ class Sample(google.protobuf.message.Message): global___Sample = Sample +@typing_extensions.final class Range(google.protobuf.message.Message): """Relation of type [[Range]] that generates a sequence of integers.""" @@ -1671,6 +1705,7 @@ class Range(google.protobuf.message.Message): global___Range = Range +@typing_extensions.final class SubqueryAlias(google.protobuf.message.Message): """Relation alias.""" @@ -1708,6 +1743,7 @@ class SubqueryAlias(google.protobuf.message.Message): global___SubqueryAlias = SubqueryAlias +@typing_extensions.final class Repartition(google.protobuf.message.Message): """Relation repartition.""" @@ -1755,6 +1791,7 @@ class Repartition(google.protobuf.message.Message): global___Repartition = Repartition +@typing_extensions.final class ShowString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.showString' to compute the results. @@ -1804,6 +1841,7 @@ class ShowString(google.protobuf.message.Message): global___ShowString = ShowString +@typing_extensions.final class StatSummary(google.protobuf.message.Message): """Computes specified statistics for numeric and string columns. It will invoke 'Dataset.summary' (same as 'StatFunctions.summary') @@ -1851,6 +1889,7 @@ class StatSummary(google.protobuf.message.Message): global___StatSummary = StatSummary +@typing_extensions.final class StatDescribe(google.protobuf.message.Message): """Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or @@ -1884,6 +1923,7 @@ class StatDescribe(google.protobuf.message.Message): global___StatDescribe = StatDescribe +@typing_extensions.final class StatCrosstab(google.protobuf.message.Message): """Computes a pair-wise frequency table of the given columns. Also known as a contingency table. It will invoke 'Dataset.stat.crosstab' (same as 'StatFunctions.crossTabulate') @@ -1925,6 +1965,7 @@ class StatCrosstab(google.protobuf.message.Message): global___StatCrosstab = StatCrosstab +@typing_extensions.final class StatCov(google.protobuf.message.Message): """Calculate the sample covariance of two numerical columns of a DataFrame. It will invoke 'Dataset.stat.cov' (same as 'StatFunctions.calculateCov') to compute the results. @@ -1959,6 +2000,7 @@ class StatCov(google.protobuf.message.Message): global___StatCov = StatCov +@typing_extensions.final class StatCorr(google.protobuf.message.Message): """Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson Correlation Coefficient. It will invoke 'Dataset.stat.corr' (same as @@ -2018,6 +2060,7 @@ class StatCorr(google.protobuf.message.Message): global___StatCorr = StatCorr +@typing_extensions.final class StatApproxQuantile(google.protobuf.message.Message): """Calculates the approximate quantiles of numerical columns of a DataFrame. It will invoke 'Dataset.stat.approxQuantile' (same as 'StatFunctions.approxQuantile') @@ -2080,6 +2123,7 @@ class StatApproxQuantile(google.protobuf.message.Message): global___StatApproxQuantile = StatApproxQuantile +@typing_extensions.final class StatFreqItems(google.protobuf.message.Message): """Finding frequent items for columns, possibly with false positives. It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems') @@ -2128,6 +2172,7 @@ class StatFreqItems(google.protobuf.message.Message): global___StatFreqItems = StatFreqItems +@typing_extensions.final class StatSampleBy(google.protobuf.message.Message): """Returns a stratified sample without replacement based on the fraction given on each stratum. @@ -2137,6 +2182,7 @@ class StatSampleBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Fraction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2218,6 +2264,7 @@ class StatSampleBy(google.protobuf.message.Message): global___StatSampleBy = StatSampleBy +@typing_extensions.final class NAFill(google.protobuf.message.Message): """Replaces null values. It will invoke 'Dataset.na.fill' (same as 'DataFrameNaFunctions.fill') to compute the results. @@ -2276,6 +2323,7 @@ class NAFill(google.protobuf.message.Message): global___NAFill = NAFill +@typing_extensions.final class NADrop(google.protobuf.message.Message): """Drop rows containing null values. It will invoke 'Dataset.na.drop' (same as 'DataFrameNaFunctions.drop') to compute the results. @@ -2344,6 +2392,7 @@ class NADrop(google.protobuf.message.Message): global___NADrop = NADrop +@typing_extensions.final class NAReplace(google.protobuf.message.Message): """Replaces old values with the corresponding values. It will invoke 'Dataset.na.replace' (same as 'DataFrameNaFunctions.replace') @@ -2352,6 +2401,7 @@ class NAReplace(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Replacement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2428,6 +2478,7 @@ class NAReplace(google.protobuf.message.Message): global___NAReplace = NAReplace +@typing_extensions.final class ToDF(google.protobuf.message.Message): """Rename columns on the input relation by the same length of names.""" @@ -2463,11 +2514,13 @@ class ToDF(google.protobuf.message.Message): global___ToDF = ToDF +@typing_extensions.final class WithColumnsRenamed(google.protobuf.message.Message): """Rename columns on the input relation by a map with name to name mapping.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class RenameColumnsMapEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2519,6 +2572,7 @@ class WithColumnsRenamed(google.protobuf.message.Message): global___WithColumnsRenamed = WithColumnsRenamed +@typing_extensions.final class WithColumns(google.protobuf.message.Message): """Adding columns or replacing the existing columns that have the same names.""" @@ -2563,6 +2617,7 @@ class WithColumns(google.protobuf.message.Message): global___WithColumns = WithColumns +@typing_extensions.final class WithWatermark(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2595,6 +2650,7 @@ class WithWatermark(google.protobuf.message.Message): global___WithWatermark = WithWatermark +@typing_extensions.final class Hint(google.protobuf.message.Message): """Specify a hint over a relation. Hint should have a name and optional parameters.""" @@ -2640,11 +2696,13 @@ class Hint(google.protobuf.message.Message): global___Hint = Hint +@typing_extensions.final class Unpivot(google.protobuf.message.Message): """Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Values(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2726,6 +2784,7 @@ class Unpivot(google.protobuf.message.Message): global___Unpivot = Unpivot +@typing_extensions.final class ToSchema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2755,6 +2814,7 @@ class ToSchema(google.protobuf.message.Message): global___ToSchema = ToSchema +@typing_extensions.final class RepartitionByExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2813,6 +2873,7 @@ class RepartitionByExpression(google.protobuf.message.Message): global___RepartitionByExpression = RepartitionByExpression +@typing_extensions.final class MapPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2867,6 +2928,7 @@ class MapPartitions(google.protobuf.message.Message): global___MapPartitions = MapPartitions +@typing_extensions.final class GroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2909,6 +2971,7 @@ class GroupMap(google.protobuf.message.Message): global___GroupMap = GroupMap +@typing_extensions.final class CoGroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2980,6 +3043,84 @@ class CoGroupMap(google.protobuf.message.Message): global___CoGroupMap = CoGroupMap +@typing_extensions.final +class ApplyInPandasWithState(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + INPUT_FIELD_NUMBER: builtins.int + GROUPING_EXPRESSIONS_FIELD_NUMBER: builtins.int + FUNC_FIELD_NUMBER: builtins.int + OUTPUT_SCHEMA_FIELD_NUMBER: builtins.int + STATE_SCHEMA_FIELD_NUMBER: builtins.int + OUTPUT_MODE_FIELD_NUMBER: builtins.int + TIMEOUT_CONF_FIELD_NUMBER: builtins.int + @property + def input(self) -> global___Relation: ... + @property + def grouping_expressions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: ... + @property + def func(self) -> pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction: ... + @property + def output_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... + @property + def state_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... + output_mode: builtins.str + timeout_conf: builtins.str + def __init__( + self, + *, + input: global___Relation | None = ..., + grouping_expressions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + func: pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction + | None = ..., + output_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., + state_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., + output_mode: builtins.str = ..., + timeout_conf: builtins.str = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "func", + b"func", + "input", + b"input", + "output_schema", + b"output_schema", + "state_schema", + b"state_schema", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "func", + b"func", + "grouping_expressions", + b"grouping_expressions", + "input", + b"input", + "output_mode", + b"output_mode", + "output_schema", + b"output_schema", + "state_schema", + b"state_schema", + "timeout_conf", + b"timeout_conf", + ], + ) -> None: ... + +global___ApplyInPandasWithState = ApplyInPandasWithState + +@typing_extensions.final class CollectMetrics(google.protobuf.message.Message): """Collect arbitrary (named) metrics from a dataset.""" @@ -3020,6 +3161,7 @@ class CollectMetrics(google.protobuf.message.Message): global___CollectMetrics = CollectMetrics +@typing_extensions.final class Parse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3041,6 +3183,7 @@ class Parse(google.protobuf.message.Message): PARSE_FORMAT_CSV: Parse.ParseFormat.ValueType # 1 PARSE_FORMAT_JSON: Parse.ParseFormat.ValueType # 2 + @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index 956701b4c366..c4b17db1abf0 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -48,6 +48,7 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor +@typing_extensions.final class DataType(google.protobuf.message.Message): """This message describes the logical [[DataType]] of something. It does not carry the value itself but only describes it. @@ -55,6 +56,7 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor + @typing_extensions.final class Boolean(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -72,6 +74,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Byte(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -89,6 +92,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Short(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -106,6 +110,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Integer(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -123,6 +128,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Long(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -140,6 +146,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Float(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -157,6 +164,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Double(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -174,6 +182,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class String(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -191,6 +200,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Binary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -208,6 +218,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class NULL(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -225,6 +236,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Timestamp(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -242,6 +254,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Date(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -259,6 +272,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class TimestampNTZ(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -276,6 +290,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -293,6 +308,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class YearMonthInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -346,6 +362,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... + @typing_extensions.final class DayTimeInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -399,6 +416,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... + @typing_extensions.final class Char(google.protobuf.message.Message): """Start compound types.""" @@ -421,6 +439,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class VarChar(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -441,6 +460,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -494,6 +514,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... + @typing_extensions.final class StructField(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -539,6 +560,7 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... + @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -564,6 +586,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -596,6 +619,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -637,6 +661,7 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + @typing_extensions.final class UDT(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -716,6 +741,7 @@ class DataType(google.protobuf.message.Message): ], ) -> typing_extensions.Literal["serialized_python_class"] | None: ... + @typing_extensions.final class Unparsed(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From 4d9429e329d3cfe17c62108e8b3460d0e6cc122d Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Thu, 6 Apr 2023 13:30:27 -0700 Subject: [PATCH 02/12] change schema type to string --- .../protobuf/spark/connect/relations.proto | 4 +-- .../connect/planner/SparkConnectPlanner.scala | 10 +++--- python/pyspark/sql/connect/group.py | 36 ++++++++++--------- python/pyspark/sql/connect/plan.py | 10 +++--- .../sql/connect/proto/relations_pb2.py | 16 ++++----- .../sql/connect/proto/relations_pb2.pyi | 22 +++--------- 6 files changed, 42 insertions(+), 56 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 3402e44c8127..30bdb4c67025 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -848,9 +848,9 @@ message ApplyInPandasWithState { CommonInlineUserDefinedFunction func = 3; - DataType output_schema = 4; + string output_schema = 4; - DataType state_schema = 5; + string state_schema = 5; string output_mode = 6; diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 471237cf820e..fb537f065577 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -590,17 +590,15 @@ class SparkConnectPlanner(val session: SparkSession) { val cols = rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) - val outputSchema = transformDataType(rel.getOutputSchema) - assert(outputSchema.isInstanceOf[StructType]) + val outputSchema = parseSchema(rel.getOutputSchema) - val stateSchema = transformDataType(rel.getStateSchema) - assert(stateSchema.isInstanceOf[StructType]) + val stateSchema = parseSchema(rel.getStateSchema) Dataset .ofRows(session, transformRelation(rel.getInput)) .groupBy(cols: _*) - .applyInPandasWithState(pythonUdf, outputSchema.asInstanceOf[StructType], - stateSchema.asInstanceOf[StructType], rel.getOutputMode, rel.getTimeoutConf) + .applyInPandasWithState(pythonUdf, outputSchema, stateSchema, rel.getOutputMode, + rel.getTimeoutConf) .logicalPlan } diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index ce7dff8c19f1..5b18140a7a3b 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -37,6 +37,7 @@ from pyspark.sql.pandas.group_ops import PandasCogroupedOps as PySparkPandasCogroupedOps from pyspark.sql.types import DataType from pyspark.sql.types import NumericType +from pyspark.sql.types import StructType import pyspark.sql.connect.plan as plan from pyspark.sql.connect.column import Column @@ -491,38 +492,39 @@ def applyInPandas( applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ - def applyInPandasWithState(self, func: "PandasGroupedMapFunction", - output_schema: Union["StructType", str], - state_schema: Union["StructType", str], - output_mode: str, - timeout_conf: str) -> "DataFrame": + def applyInPandasWithState( + self, + func: "PandasGroupedMapFunctionWithState", + outputStructType: Union[StructType, str], + stateStructType: Union[StructType, str], + outputMode: str, + timeoutConf: str + ) -> "DataFrame": from pyspark.sql.connect.udf import UserDefinedFunction from pyspark.sql.connect.dataframe import DataFrame - from pyspark.sql.connect.types import UnparsedDataType udf_obj = UserDefinedFunction( func, - returnType=output_schema, + returnType=outputStructType, evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE, ) - output_type: DataType = ( - UnparsedDataType(output_schema) if isinstance(output_schema, str) else output_schema - ) + # TODO: raise type error + output_schema: str = outputStructType.json() if isinstance(outputStructType, StructType)\ + else outputStructType - state_type: DataType = ( - UnparsedDataType(state_schema) if isinstance(state_schema, str) else state_schema - ) + state_schema: str = stateStructType.json() if isinstance(stateStructType, StructType) \ + else stateStructType return DataFrame.withPlan( plan.ApplyInPandasWithState( child=self._df._plan, grouping_cols=self._grouping_cols, function=udf_obj, - output_schema=output_type, - state_schema=state_type, - output_mode=output_mode, - timeout_conf=timeout_conf, + output_schema=output_schema, + state_schema=state_schema, + output_mode=outputMode, + timeout_conf=timeoutConf, cols=self._df.columns, ), session=self._df._session, diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index e841934490b0..fdab537c3c1e 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2052,8 +2052,8 @@ def __init__( child: Optional["LogicalPlan"], grouping_cols: Sequence[Column], function: "UserDefinedFunction", - output_schema: DataType, - state_schema: DataType, + output_schema: str, + state_schema: str, output_mode: str, timeout_conf: str, cols: List[str], @@ -2076,10 +2076,8 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: [c.to_plan(session) for c in self._grouping_cols] ) plan.apply_in_pandas_with_state.func.CopyFrom(self._func.to_plan_udf(session)) - plan.apply_in_pandas_with_state.output_schema \ - .CopyFrom(pyspark_types_to_proto_types(self._output_schema)) - plan.apply_in_pandas_with_state.state_schema \ - .CopyFrom(pyspark_types_to_proto_types(self._state_schema)) + plan.apply_in_pandas_with_state.output_schema = self._output_schema + plan.apply_in_pandas_with_state.state_schema = self._state_schema plan.apply_in_pandas_with_state.output_mode = self._output_mode plan.apply_in_pandas_with_state.timeout_conf = self._timeout_conf return plan diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 97afa9f503e8..8229dc18afef 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -36,7 +36,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"\xdb\x15\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\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\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"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xa9\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\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"\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"\xd7\x03\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"\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"\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"\xc6\x04\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\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"\x81\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"\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"\xab\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\x42\x16\n\x14_all_columns_as_keys"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"\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"\\\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"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\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"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"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"\xb5\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\x42\r\n\x0b_is_barrier"\xcb\x01\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"\xe0\x02\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"\x97\x03\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(\x0b\x32\x17.spark.connect.DataTypeR\x0coutputSchema\x12:\n\x0cstate_schema\x18\x05 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\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_schemaB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\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"\xdb\x15\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\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\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"\t\n\x07Unknown"[\n\x0eRelationCommon\x12\x1f\n\x0bsource_info\x18\x01 \x01(\tR\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id"\xa9\x01\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x30\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryR\x04\x61rgs\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"\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"\xd7\x03\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"\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"\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"\xc6\x04\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\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"\x81\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"\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"\xab\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\x42\x16\n\x14_all_columns_as_keys"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"\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"\\\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"\xef\x01\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x65\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryR\x10renameColumnsMap\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"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"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"\xb5\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\x42\r\n\x0b_is_barrier"\xcb\x01\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"\xe0\x02\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"\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"\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_schemaB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) @@ -860,13 +860,13 @@ _COGROUPMAP._serialized_start = 10757 _COGROUPMAP._serialized_end = 11109 _APPLYINPANDASWITHSTATE._serialized_start = 11112 - _APPLYINPANDASWITHSTATE._serialized_end = 11519 - _COLLECTMETRICS._serialized_start = 11522 - _COLLECTMETRICS._serialized_end = 11658 - _PARSE._serialized_start = 11661 - _PARSE._serialized_end = 12049 + _APPLYINPANDASWITHSTATE._serialized_end = 11469 + _COLLECTMETRICS._serialized_start = 11472 + _COLLECTMETRICS._serialized_end = 11608 + _PARSE._serialized_start = 11611 + _PARSE._serialized_end = 11999 _PARSE_OPTIONSENTRY._serialized_start = 3535 _PARSE_OPTIONSENTRY._serialized_end = 3593 - _PARSE_PARSEFORMAT._serialized_start = 11950 - _PARSE_PARSEFORMAT._serialized_end = 12038 + _PARSE_PARSEFORMAT._serialized_start = 11900 + _PARSE_PARSEFORMAT._serialized_end = 11988 # @@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 d91375ebb78e..fdbc56978cbe 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -3064,10 +3064,8 @@ class ApplyInPandasWithState(google.protobuf.message.Message): ]: ... @property def func(self) -> pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction: ... - @property - def output_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... - @property - def state_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... + output_schema: builtins.str + state_schema: builtins.str output_mode: builtins.str timeout_conf: builtins.str def __init__( @@ -3080,23 +3078,13 @@ class ApplyInPandasWithState(google.protobuf.message.Message): | None = ..., func: pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction | None = ..., - output_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., - state_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., + output_schema: builtins.str = ..., + state_schema: builtins.str = ..., output_mode: builtins.str = ..., timeout_conf: builtins.str = ..., ) -> None: ... def HasField( - self, - field_name: typing_extensions.Literal[ - "func", - b"func", - "input", - b"input", - "output_schema", - b"output_schema", - "state_schema", - b"state_schema", - ], + self, field_name: typing_extensions.Literal["func", b"func", "input", b"input"] ) -> builtins.bool: ... def ClearField( self, From b1d64717a473595061b39ea116dfe86769c13fcc Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Thu, 6 Apr 2023 13:34:13 -0700 Subject: [PATCH 03/12] remove unused code --- python/pyspark/sql/connect/group.py | 228 ---------------------------- 1 file changed, 228 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 5b18140a7a3b..973ba819fb5a 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -264,234 +264,6 @@ def applyInPandas( applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ - -from pyspark.sql.types import NumericType - -import pyspark.sql.connect.plan as plan -from pyspark.sql.connect.column import Column -from pyspark.sql.connect.functions import _invoke_function, col, lit - -if TYPE_CHECKING: - from pyspark.sql.connect._typing import ( - LiteralType, - PandasGroupedMapFunction, - GroupedMapPandasUserDefinedFunction, - PandasCogroupedMapFunction, - ) - from pyspark.sql.connect.dataframe import DataFrame - from pyspark.sql.types import StructType - - -class GroupedData: - def __init__( - self, - df: "DataFrame", - group_type: str, - grouping_cols: Sequence["Column"], - pivot_col: Optional["Column"] = None, - pivot_values: Optional[Sequence["LiteralType"]] = None, - ) -> None: - from pyspark.sql.connect.dataframe import DataFrame - - assert isinstance(df, DataFrame) - self._df = df - - assert isinstance(group_type, str) and group_type in ["groupby", "rollup", "cube", "pivot"] - self._group_type = group_type - - assert isinstance(grouping_cols, list) and all(isinstance(g, Column) for g in grouping_cols) - self._grouping_cols: List[Column] = grouping_cols - - self._pivot_col: Optional["Column"] = None - self._pivot_values: Optional[List[Any]] = None - if group_type == "pivot": - assert pivot_col is not None and isinstance(pivot_col, Column) - assert pivot_values is None or isinstance(pivot_values, list) - self._pivot_col = pivot_col - self._pivot_values = pivot_values - - @overload - def agg(self, *exprs: Column) -> "DataFrame": - ... - - @overload - def agg(self, __exprs: Dict[str, str]) -> "DataFrame": - ... - - def agg(self, *exprs: Union[Column, Dict[str, str]]) -> "DataFrame": - from pyspark.sql.connect.dataframe import DataFrame - - assert exprs, "exprs should not be empty" - if len(exprs) == 1 and isinstance(exprs[0], dict): - # Convert the dict into key value pairs - aggregate_cols = [_invoke_function(exprs[0][k], col(k)) for k in exprs[0]] - else: - # Columns - assert all(isinstance(c, Column) for c in exprs), "all exprs should be Column" - aggregate_cols = cast(List[Column], list(exprs)) - - return DataFrame.withPlan( - plan.Aggregate( - child=self._df._plan, - group_type=self._group_type, - grouping_cols=self._grouping_cols, - aggregate_cols=aggregate_cols, - pivot_col=self._pivot_col, - pivot_values=self._pivot_values, - ), - session=self._df._session, - ) - - agg.__doc__ = PySparkGroupedData.agg.__doc__ - - def _numeric_agg(self, function: str, cols: Sequence[str]) -> "DataFrame": - from pyspark.sql.connect.dataframe import DataFrame - - assert isinstance(function, str) and function in ["min", "max", "avg", "sum"] - - assert isinstance(cols, list) and all(isinstance(c, str) for c in cols) - - schema = self._df.schema - - numerical_cols: List[str] = [ - field.name for field in schema.fields if isinstance(field.dataType, NumericType) - ] - - agg_cols: List[str] = [] - - if len(cols) > 0: - invalid_cols = [c for c in cols if c not in numerical_cols] - if len(invalid_cols) > 0: - raise TypeError( - f"{invalid_cols} are not numeric columns. " - f"Numeric aggregation function can only be applied on numeric columns." - ) - agg_cols = cols - else: - # if no column is provided, then all numerical columns are selected - agg_cols = numerical_cols - - return DataFrame.withPlan( - plan.Aggregate( - child=self._df._plan, - group_type=self._group_type, - grouping_cols=self._grouping_cols, - aggregate_cols=[_invoke_function(function, col(c)) for c in agg_cols], - pivot_col=self._pivot_col, - pivot_values=self._pivot_values, - ), - session=self._df._session, - ) - - def min(self, *cols: str) -> "DataFrame": - return self._numeric_agg("min", list(cols)) - - min.__doc__ = PySparkGroupedData.min.__doc__ - - def max(self, *cols: str) -> "DataFrame": - return self._numeric_agg("max", list(cols)) - - max.__doc__ = PySparkGroupedData.max.__doc__ - - def sum(self, *cols: str) -> "DataFrame": - return self._numeric_agg("sum", list(cols)) - - sum.__doc__ = PySparkGroupedData.sum.__doc__ - - def avg(self, *cols: str) -> "DataFrame": - return self._numeric_agg("avg", list(cols)) - - avg.__doc__ = PySparkGroupedData.avg.__doc__ - - mean = avg - - def count(self) -> "DataFrame": - return self.agg(_invoke_function("count", lit(1)).alias("count")) - - count.__doc__ = PySparkGroupedData.count.__doc__ - - def pivot(self, pivot_col: str, values: Optional[List["LiteralType"]] = None) -> "GroupedData": - if self._group_type != "groupby": - if self._group_type == "pivot": - raise Exception("Repeated PIVOT operation is not supported!") - else: - raise Exception(f"PIVOT after {self._group_type.upper()} is not supported!") - - if not isinstance(pivot_col, str): - raise TypeError( - f"pivot_col should be a str, but got {type(pivot_col).__name__} {pivot_col}" - ) - - if values is not None: - if not isinstance(values, list): - raise TypeError( - f"values should be a list, but got {type(values).__name__} {values}" - ) - for v in values: - if not isinstance(v, (bool, float, int, str)): - raise TypeError( - f"value should be a bool, float, int or str, but got {type(v).__name__} {v}" - ) - - return GroupedData( - df=self._df, - group_type="pivot", - grouping_cols=self._grouping_cols, - pivot_col=self._df[pivot_col], - pivot_values=values, - ) - - pivot.__doc__ = PySparkGroupedData.pivot.__doc__ - - def apply(self, udf: "GroupedMapPandasUserDefinedFunction") -> "DataFrame": - # Columns are special because hasattr always return True - if ( - isinstance(udf, Column) - or not hasattr(udf, "func") - or ( - udf.evalType # type: ignore[attr-defined] - != PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF - ) - ): - raise ValueError( - "Invalid udf: the udf argument must be a pandas_udf of type " "GROUPED_MAP." - ) - - warnings.warn( - "It is preferred to use 'applyInPandas' over this " - "API. This API will be deprecated in the future releases. See SPARK-28264 for " - "more details.", - UserWarning, - ) - - return self.applyInPandas(udf.func, schema=udf.returnType) # type: ignore[attr-defined] - - apply.__doc__ = PySparkGroupedData.apply.__doc__ - - def applyInPandas( - self, func: "PandasGroupedMapFunction", schema: Union["StructType", str] - ) -> "DataFrame": - from pyspark.sql.connect.udf import UserDefinedFunction - from pyspark.sql.connect.dataframe import DataFrame - - udf_obj = UserDefinedFunction( - func, - returnType=schema, - evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF, - ) - - return DataFrame.withPlan( - plan.GroupMap( - child=self._df._plan, - grouping_cols=self._grouping_cols, - function=udf_obj, - cols=self._df.columns, - ), - session=self._df._session, - ) - - applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ - def applyInPandasWithState( self, func: "PandasGroupedMapFunctionWithState", From c0b32a305f949c001f1cfdeb567ff590c5cc4bb0 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Mon, 10 Apr 2023 22:06:14 -0700 Subject: [PATCH 04/12] format --- .../protobuf/spark/connect/relations.proto | 7 +++++++ python/pyspark/sql/connect/group.py | 1 - python/pyspark/sql/connect/plan.py | 18 +++++++++--------- 3 files changed, 16 insertions(+), 10 deletions(-) diff --git a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto index 30bdb4c67025..f49fca079b03 100644 --- a/connector/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/connector/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -842,18 +842,25 @@ message CoGroupMap { } message ApplyInPandasWithState { + // (Required) Input relation for applyInPandasWithState. Relation input = 1; + // (Required) Expressions for grouping keys. repeated Expression grouping_expressions = 2; + // (Required) Input user-defined function. CommonInlineUserDefinedFunction func = 3; + // (Required) Schema for the output DataFrame. string output_schema = 4; + // (Required) Schema for the state. string state_schema = 5; + // (Required) The output mode of the function. string output_mode = 6; + // (Required) Timeout configuration for groups that do not receive data for a while. string timeout_conf = 7; } diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 973ba819fb5a..e1e9ffbfbaae 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -281,7 +281,6 @@ def applyInPandasWithState( evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE, ) - # TODO: raise type error output_schema: str = outputStructType.json() if isinstance(outputStructType, StructType)\ else outputStructType diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index fdab537c3c1e..86c32600095a 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2048,15 +2048,15 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: class ApplyInPandasWithState(LogicalPlan): """Logical plan object for a applyInPandasWithState.""" def __init__( - self, - child: Optional["LogicalPlan"], - grouping_cols: Sequence[Column], - function: "UserDefinedFunction", - output_schema: str, - state_schema: str, - output_mode: str, - timeout_conf: str, - cols: List[str], + self, + child: Optional["LogicalPlan"], + grouping_cols: Sequence[Column], + function: "UserDefinedFunction", + output_schema: str, + state_schema: str, + output_mode: str, + timeout_conf: str, + cols: List[str], ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) From 2099668c70aaf688c81cd4e9781623668ddde183 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Mon, 10 Apr 2023 22:41:52 -0700 Subject: [PATCH 05/12] proto change --- python/pyspark/sql/connect/proto/relations_pb2.pyi | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index fdbc56978cbe..39e2e5237e4f 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -3055,19 +3055,26 @@ class ApplyInPandasWithState(google.protobuf.message.Message): OUTPUT_MODE_FIELD_NUMBER: builtins.int TIMEOUT_CONF_FIELD_NUMBER: builtins.int @property - def input(self) -> global___Relation: ... + def input(self) -> global___Relation: + """(Required) Input relation for applyInPandasWithState.""" @property def grouping_expressions( self, ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ pyspark.sql.connect.proto.expressions_pb2.Expression - ]: ... + ]: + """(Required) Expressions for grouping keys.""" @property - def func(self) -> pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction: ... + def func(self) -> pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction: + """(Required) Input user-defined function.""" output_schema: builtins.str + """(Required) Schema for the output DataFrame.""" state_schema: builtins.str + """(Required) Schema for the state.""" output_mode: builtins.str + """(Required) The output mode of the function.""" timeout_conf: builtins.str + """(Required) Timeout configuration for groups that do not receive data for a while.""" def __init__( self, *, From 85be6c16acaf6a0b6223bd62368baefd8949cd2c Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Wed, 12 Apr 2023 10:20:33 -0700 Subject: [PATCH 06/12] add test file --- dev/sparktestsupport/modules.py | 1 + ...st_parity_pandas_grouped_map_with_state.py | 42 +++++++++++++++++++ .../test_pandas_grouped_map_with_state.py | 6 ++- 3 files changed, 48 insertions(+), 1 deletion(-) create mode 100644 python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 4f6f7765a5b7..5df3a536f8f6 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -778,6 +778,7 @@ def __hash__(self): "pyspark.sql.tests.connect.test_parity_pandas_grouped_map", "pyspark.sql.tests.connect.test_parity_pandas_cogrouped_map", "pyspark.sql.tests.connect.streaming.test_parity_streaming", + "pyspark.sql.tests.connect.test_parity_pandas_grouped_map_with_state", # ml doctests "pyspark.ml.connect.functions", # ml unittests diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py new file mode 100644 index 000000000000..3d437e0deaf6 --- /dev/null +++ b/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py @@ -0,0 +1,42 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +import unittest + +from pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state import GroupedApplyInPandasWithStateTestsMixin +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class GroupedApplyInPandasWithStateTests(GroupedApplyInPandasWithStateTestsMixin, ReusedConnectTestCase): + @unittest.skip("Query manager API will be supported later with SPARK-43032.") + def test_apply_in_pandas_with_state_basic(self): + super().test_apply_in_pandas_with_state_basic() + + @unittest.skip("Query manager API will be supported later with SPARK-43032.") + def test_apply_in_pandas_with_state_python_worker_random_failure(self): + super().test_apply_in_pandas_with_state_python_worker_random_failure() + + +if __name__ == "__main__": + from pyspark.sql.tests.connect.test_parity_pandas_grouped_map_with_state import * # noqa: F401 + + try: + import xmlrunner + + testRunner = xmlrunner.XMLTestRunner(output="target/test-reports", verbosity=2) + except ImportError: + testRunner = None + unittest.main(testRunner=testRunner, verbosity=2) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py index 9600d1e34457..dce016ebeaaf 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py @@ -53,7 +53,7 @@ not have_pandas or not have_pyarrow, cast(str, pandas_requirement_message or pyarrow_requirement_message), ) -class GroupedApplyInPandasWithStateTests(ReusedSQLTestCase): +class GroupedApplyInPandasWithStateTestsMixin: @classmethod def conf(cls): cfg = SparkConf() @@ -236,6 +236,10 @@ def assert_test(): q.stop() +class GroupedApplyInPandasWithStateTests(GroupedApplyInPandasWithStateTestsMixin, ReusedSQLTestCase): + pass + + if __name__ == "__main__": from pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state import * # noqa: F401 From e4fe03d42fe5d55eaed8669810d1f94f01e12b66 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Wed, 12 Apr 2023 10:51:12 -0700 Subject: [PATCH 07/12] add versionchanged --- python/pyspark/sql/pandas/group_ops.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 12ceb56c79ff..857ecd2e5821 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -273,6 +273,9 @@ def applyInPandasWithState( .. versionadded:: 3.4.0 + .. versionchanged:: 3.5.0 + Supports Spark Connect. + Parameters ---------- func : function From 4878b5b91523f9c903f6b7c5558b45daa6f59d7f Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Thu, 13 Apr 2023 12:03:11 -0700 Subject: [PATCH 08/12] remove unsupported group function test --- python/pyspark/sql/tests/connect/test_connect_basic.py | 7 ------- 1 file changed, 7 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 166a8b23bd71..b1b3a94accff 100644 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -2928,13 +2928,6 @@ def test_unsupported_functions(self): with self.assertRaises(NotImplementedError): getattr(df, f)() - def test_unsupported_group_functions(self): - # SPARK-41927: Disable unsupported functions. - cg = self.connect.read.table(self.tbl_name).groupBy("id") - for f in ("applyInPandasWithState",): - with self.assertRaises(NotImplementedError): - getattr(cg, f)() - def test_unsupported_session_functions(self): # SPARK-41934: Disable unsupported functions. From 3453ce248ef2ec0426f0fec5246d48d09e4d3250 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Thu, 13 Apr 2023 12:08:11 -0700 Subject: [PATCH 09/12] formatting --- .../spark/sql/connect/planner/SparkConnectPlanner.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index fb537f065577..7b46d8033fc3 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -597,7 +597,11 @@ class SparkConnectPlanner(val session: SparkSession) { Dataset .ofRows(session, transformRelation(rel.getInput)) .groupBy(cols: _*) - .applyInPandasWithState(pythonUdf, outputSchema, stateSchema, rel.getOutputMode, + .applyInPandasWithState( + pythonUdf, + outputSchema, + stateSchema, + rel.getOutputMode, rel.getTimeoutConf) .logicalPlan } From 2190975ed10ac2a29fef845dad0f2967be591d66 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Fri, 14 Apr 2023 10:16:37 -0700 Subject: [PATCH 10/12] python format --- python/pyspark/sql/connect/group.py | 22 +++++++++++-------- python/pyspark/sql/connect/plan.py | 1 + ...st_parity_pandas_grouped_map_with_state.py | 8 +++++-- .../test_pandas_grouped_map_with_state.py | 4 +++- 4 files changed, 23 insertions(+), 12 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index e1e9ffbfbaae..276712583e9a 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -265,12 +265,12 @@ def applyInPandas( applyInPandas.__doc__ = PySparkGroupedData.applyInPandas.__doc__ def applyInPandasWithState( - self, - func: "PandasGroupedMapFunctionWithState", - outputStructType: Union[StructType, str], - stateStructType: Union[StructType, str], - outputMode: str, - timeoutConf: str + self, + func: "PandasGroupedMapFunctionWithState", + outputStructType: Union[StructType, str], + stateStructType: Union[StructType, str], + outputMode: str, + timeoutConf: str, ) -> "DataFrame": from pyspark.sql.connect.udf import UserDefinedFunction from pyspark.sql.connect.dataframe import DataFrame @@ -281,11 +281,15 @@ def applyInPandasWithState( evalType=PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF_WITH_STATE, ) - output_schema: str = outputStructType.json() if isinstance(outputStructType, StructType)\ + output_schema: str = ( + outputStructType.json() + if isinstance(outputStructType, StructType) else outputStructType + ) - state_schema: str = stateStructType.json() if isinstance(stateStructType, StructType) \ - else stateStructType + state_schema: str = ( + stateStructType.json() if isinstance(stateStructType, StructType) else stateStructType + ) return DataFrame.withPlan( plan.ApplyInPandasWithState( diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 86c32600095a..f39faee107c9 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2047,6 +2047,7 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: class ApplyInPandasWithState(LogicalPlan): """Logical plan object for a applyInPandasWithState.""" + def __init__( self, child: Optional["LogicalPlan"], diff --git a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py index 3d437e0deaf6..685f12a0c825 100644 --- a/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py +++ b/python/pyspark/sql/tests/connect/test_parity_pandas_grouped_map_with_state.py @@ -16,11 +16,15 @@ # import unittest -from pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state import GroupedApplyInPandasWithStateTestsMixin +from pyspark.sql.tests.pandas.test_pandas_grouped_map_with_state import ( + GroupedApplyInPandasWithStateTestsMixin, +) from pyspark.testing.connectutils import ReusedConnectTestCase -class GroupedApplyInPandasWithStateTests(GroupedApplyInPandasWithStateTestsMixin, ReusedConnectTestCase): +class GroupedApplyInPandasWithStateTests( + GroupedApplyInPandasWithStateTestsMixin, ReusedConnectTestCase +): @unittest.skip("Query manager API will be supported later with SPARK-43032.") def test_apply_in_pandas_with_state_basic(self): super().test_apply_in_pandas_with_state_basic() diff --git a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py index dce016ebeaaf..a2a6544faa0f 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_grouped_map_with_state.py @@ -236,7 +236,9 @@ def assert_test(): q.stop() -class GroupedApplyInPandasWithStateTests(GroupedApplyInPandasWithStateTestsMixin, ReusedSQLTestCase): +class GroupedApplyInPandasWithStateTests( + GroupedApplyInPandasWithStateTestsMixin, ReusedSQLTestCase +): pass From 393d2de59b51c5b97b862b186be46fcb1fb3b0be Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Sun, 16 Apr 2023 13:26:30 -0700 Subject: [PATCH 11/12] format --- dev/tox.ini | 1 + python/pyspark/sql/connect/_typing.py | 5 ++ python/pyspark/sql/connect/group.py | 2 +- python/pyspark/sql/connect/proto/base_pb2.pyi | 57 ------------------ .../pyspark/sql/connect/proto/catalog_pb2.pyi | 29 ---------- .../sql/connect/proto/commands_pb2.pyi | 21 ------- .../pyspark/sql/connect/proto/common_pb2.pyi | 1 - .../sql/connect/proto/example_plugins_pb2.pyi | 3 - .../sql/connect/proto/expressions_pb2.pyi | 24 -------- .../sql/connect/proto/relations_pb2.pyi | 58 ------------------- .../pyspark/sql/connect/proto/types_pb2.pyi | 26 --------- 11 files changed, 7 insertions(+), 220 deletions(-) diff --git a/dev/tox.ini b/dev/tox.ini index c6edee272add..2751c5521cc6 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -57,4 +57,5 @@ exclude = python/pyspark/java_gateway.pyi, dev/ansible-for-test-node/*, python/pyspark/sql/connect/proto/*, + */venv/* max-line-length = 100 diff --git a/python/pyspark/sql/connect/_typing.py b/python/pyspark/sql/connect/_typing.py index 6bdde2926cbd..4c76e37659c0 100644 --- a/python/pyspark/sql/connect/_typing.py +++ b/python/pyspark/sql/connect/_typing.py @@ -32,6 +32,7 @@ from pyspark.sql.connect.column import Column from pyspark.sql.connect.types import DataType +from pyspark.sql.streaming.state import GroupState ColumnOrName = Union[Column, str] @@ -63,6 +64,10 @@ PandasCogroupedMapFunction = Callable[[DataFrameLike, DataFrameLike], DataFrameLike] +PandasGroupedMapFunctionWithState = Callable[ + [Any, Iterable[DataFrameLike], GroupState], Iterable[DataFrameLike] +] + class UserDefinedFunctionLike(Protocol): func: Callable[..., Any] diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 276712583e9a..160b1b49072a 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -35,7 +35,6 @@ from pyspark.rdd import PythonEvalType from pyspark.sql.group import GroupedData as PySparkGroupedData from pyspark.sql.pandas.group_ops import PandasCogroupedOps as PySparkPandasCogroupedOps -from pyspark.sql.types import DataType from pyspark.sql.types import NumericType from pyspark.sql.types import StructType @@ -49,6 +48,7 @@ PandasGroupedMapFunction, GroupedMapPandasUserDefinedFunction, PandasCogroupedMapFunction, + PandasGroupedMapFunctionWithState, ) from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.types import StructType diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 9ae9a8cd249b..15e5fb53f140 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -55,7 +55,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Plan(google.protobuf.message.Message): """A [[Plan]] is the structure that carries the runtime information for the execution from the client to the server. A [[Plan]] can either be of the type [[Relation]] which is a reference @@ -95,7 +94,6 @@ class Plan(google.protobuf.message.Message): global___Plan = Plan -@typing_extensions.final class UserContext(google.protobuf.message.Message): """User Context is used to refer to one particular user session that is executing queries in the backend. @@ -135,13 +133,11 @@ class UserContext(google.protobuf.message.Message): global___UserContext = UserContext -@typing_extensions.final class AnalyzePlanRequest(google.protobuf.message.Message): """Request to perform plan analyze, optionally to explain the plan.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -159,7 +155,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class Explain(google.protobuf.message.Message): """Explains the input plan based on a configurable mode.""" @@ -234,7 +229,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): field_name: typing_extensions.Literal["explain_mode", b"explain_mode", "plan", b"plan"], ) -> None: ... - @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -252,7 +246,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -270,7 +263,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -288,7 +280,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -306,7 +297,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -314,7 +304,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -330,7 +319,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["ddl_string", b"ddl_string"] ) -> None: ... - @typing_extensions.final class SameSemantics(google.protobuf.message.Message): """Returns `true` when the logical query plans are equal and therefore return same results.""" @@ -363,7 +351,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -381,7 +368,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): ) -> builtins.bool: ... def ClearField(self, field_name: typing_extensions.Literal["plan", b"plan"]) -> None: ... - @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -425,7 +411,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_storage_level", b"_storage_level"] ) -> typing_extensions.Literal["storage_level"] | None: ... - @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -458,7 +443,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_blocking", b"_blocking"] ) -> typing_extensions.Literal["blocking"] | None: ... - @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -660,7 +644,6 @@ class AnalyzePlanRequest(google.protobuf.message.Message): global___AnalyzePlanRequest = AnalyzePlanRequest -@typing_extensions.final class AnalyzePlanResponse(google.protobuf.message.Message): """Response to performing analysis of the query. Contains relevant metadata to be able to reason about the performance. @@ -668,7 +651,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Schema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -687,7 +669,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["schema", b"schema"] ) -> None: ... - @typing_extensions.final class Explain(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -702,7 +683,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["explain_string", b"explain_string"] ) -> None: ... - @typing_extensions.final class TreeString(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -717,7 +697,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["tree_string", b"tree_string"] ) -> None: ... - @typing_extensions.final class IsLocal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -732,7 +711,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_local", b"is_local"] ) -> None: ... - @typing_extensions.final class IsStreaming(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -747,7 +725,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["is_streaming", b"is_streaming"] ) -> None: ... - @typing_extensions.final class InputFiles(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -764,7 +741,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["files", b"files"]) -> None: ... - @typing_extensions.final class SparkVersion(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -779,7 +755,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["version", b"version"] ) -> None: ... - @typing_extensions.final class DDLParse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -798,7 +773,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["parsed", b"parsed"] ) -> None: ... - @typing_extensions.final class SameSemantics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -813,7 +787,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... - @typing_extensions.final class SemanticHash(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -828,7 +801,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["result", b"result"] ) -> None: ... - @typing_extensions.final class Persist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -836,7 +808,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class Unpersist(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -844,7 +815,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): self, ) -> None: ... - @typing_extensions.final class GetStorageLevel(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1011,7 +981,6 @@ class AnalyzePlanResponse(google.protobuf.message.Message): global___AnalyzePlanResponse = AnalyzePlanResponse -@typing_extensions.final class ExecutePlanRequest(google.protobuf.message.Message): """A request to be executed by the service.""" @@ -1085,7 +1054,6 @@ class ExecutePlanRequest(google.protobuf.message.Message): global___ExecutePlanRequest = ExecutePlanRequest -@typing_extensions.final class ExecutePlanResponse(google.protobuf.message.Message): """The response of a query, can be one or more for each request. Responses belonging to the same input query, carry the same `session_id`. @@ -1093,7 +1061,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class SqlCommandResult(google.protobuf.message.Message): """A SQL command returns an opaque Relation that can be directly used as input for the next call. @@ -1116,7 +1083,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["relation", b"relation"] ) -> None: ... - @typing_extensions.final class ArrowBatch(google.protobuf.message.Message): """Batch results of metrics.""" @@ -1136,15 +1102,12 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "row_count", b"row_count"] ) -> None: ... - @typing_extensions.final class Metrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class MetricObject(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ExecutionMetricsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1204,7 +1167,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class MetricValue(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1245,7 +1207,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["metrics", b"metrics"] ) -> None: ... - @typing_extensions.final class ObservedMetrics(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1401,7 +1362,6 @@ class ExecutePlanResponse(google.protobuf.message.Message): global___ExecutePlanResponse = ExecutePlanResponse -@typing_extensions.final class KeyValue(google.protobuf.message.Message): """The key-value pair for the config request and response.""" @@ -1434,13 +1394,11 @@ class KeyValue(google.protobuf.message.Message): global___KeyValue = KeyValue -@typing_extensions.final class ConfigRequest(google.protobuf.message.Message): """Request to update or fetch the configurations.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Operation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1524,7 +1482,6 @@ class ConfigRequest(google.protobuf.message.Message): "set", "get", "get_with_default", "get_option", "get_all", "unset", "is_modifiable" ] | None: ... - @typing_extensions.final class Set(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1541,7 +1498,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... - @typing_extensions.final class Get(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1558,7 +1514,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class GetWithDefault(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1575,7 +1530,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... - @typing_extensions.final class GetOption(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1592,7 +1546,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class GetAll(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1614,7 +1567,6 @@ class ConfigRequest(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_prefix", b"_prefix"] ) -> typing_extensions.Literal["prefix"] | None: ... - @typing_extensions.final class Unset(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1631,7 +1583,6 @@ class ConfigRequest(google.protobuf.message.Message): ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["keys", b"keys"]) -> None: ... - @typing_extensions.final class IsModifiable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1712,7 +1663,6 @@ class ConfigRequest(google.protobuf.message.Message): global___ConfigRequest = ConfigRequest -@typing_extensions.final class ConfigResponse(google.protobuf.message.Message): """Response to the config request.""" @@ -1755,13 +1705,11 @@ class ConfigResponse(google.protobuf.message.Message): global___ConfigResponse = ConfigResponse -@typing_extensions.final class AddArtifactsRequest(google.protobuf.message.Message): """Request to transfer client-local artifacts.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArtifactChunk(google.protobuf.message.Message): """A chunk of an Artifact.""" @@ -1783,7 +1731,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["crc", b"crc", "data", b"data"] ) -> None: ... - @typing_extensions.final class SingleChunkArtifact(google.protobuf.message.Message): """An artifact that is contained in a single `ArtifactChunk`. Generally, this message represents tiny artifacts such as REPL-generated class files. @@ -1817,7 +1764,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["data", b"data", "name", b"name"] ) -> None: ... - @typing_extensions.final class Batch(google.protobuf.message.Message): """A number of `SingleChunkArtifact` batched into a single RPC.""" @@ -1840,7 +1786,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["artifacts", b"artifacts"] ) -> None: ... - @typing_extensions.final class BeginChunkedArtifact(google.protobuf.message.Message): """Signals the beginning/start of a chunked artifact. A large artifact is transferred through a payload of `BeginChunkedArtifact` followed by a @@ -1986,7 +1931,6 @@ class AddArtifactsRequest(google.protobuf.message.Message): global___AddArtifactsRequest = AddArtifactsRequest -@typing_extensions.final class AddArtifactsResponse(google.protobuf.message.Message): """Response to adding an artifact. Contains relevant metadata to verify successful transfer of artifact(s). @@ -1994,7 +1938,6 @@ class AddArtifactsResponse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArtifactSummary(google.protobuf.message.Message): """Metadata of an artifact.""" diff --git a/python/pyspark/sql/connect/proto/catalog_pb2.pyi b/python/pyspark/sql/connect/proto/catalog_pb2.pyi index b550cf191e1d..6e0ee5f5c311 100644 --- a/python/pyspark/sql/connect/proto/catalog_pb2.pyi +++ b/python/pyspark/sql/connect/proto/catalog_pb2.pyi @@ -50,7 +50,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Catalog(google.protobuf.message.Message): """Catalog messages are marked as unstable.""" @@ -315,7 +314,6 @@ class Catalog(google.protobuf.message.Message): global___Catalog = Catalog -@typing_extensions.final class CurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.currentDatabase`""" @@ -327,7 +325,6 @@ class CurrentDatabase(google.protobuf.message.Message): global___CurrentDatabase = CurrentDatabase -@typing_extensions.final class SetCurrentDatabase(google.protobuf.message.Message): """See `spark.catalog.setCurrentDatabase`""" @@ -345,7 +342,6 @@ class SetCurrentDatabase(google.protobuf.message.Message): global___SetCurrentDatabase = SetCurrentDatabase -@typing_extensions.final class ListDatabases(google.protobuf.message.Message): """See `spark.catalog.listDatabases`""" @@ -357,7 +353,6 @@ class ListDatabases(google.protobuf.message.Message): global___ListDatabases = ListDatabases -@typing_extensions.final class ListTables(google.protobuf.message.Message): """See `spark.catalog.listTables`""" @@ -383,7 +378,6 @@ class ListTables(google.protobuf.message.Message): global___ListTables = ListTables -@typing_extensions.final class ListFunctions(google.protobuf.message.Message): """See `spark.catalog.listFunctions`""" @@ -409,7 +403,6 @@ class ListFunctions(google.protobuf.message.Message): global___ListFunctions = ListFunctions -@typing_extensions.final class ListColumns(google.protobuf.message.Message): """See `spark.catalog.listColumns`""" @@ -442,7 +435,6 @@ class ListColumns(google.protobuf.message.Message): global___ListColumns = ListColumns -@typing_extensions.final class GetDatabase(google.protobuf.message.Message): """See `spark.catalog.getDatabase`""" @@ -460,7 +452,6 @@ class GetDatabase(google.protobuf.message.Message): global___GetDatabase = GetDatabase -@typing_extensions.final class GetTable(google.protobuf.message.Message): """See `spark.catalog.getTable`""" @@ -493,7 +484,6 @@ class GetTable(google.protobuf.message.Message): global___GetTable = GetTable -@typing_extensions.final class GetFunction(google.protobuf.message.Message): """See `spark.catalog.getFunction`""" @@ -526,7 +516,6 @@ class GetFunction(google.protobuf.message.Message): global___GetFunction = GetFunction -@typing_extensions.final class DatabaseExists(google.protobuf.message.Message): """See `spark.catalog.databaseExists`""" @@ -544,7 +533,6 @@ class DatabaseExists(google.protobuf.message.Message): global___DatabaseExists = DatabaseExists -@typing_extensions.final class TableExists(google.protobuf.message.Message): """See `spark.catalog.tableExists`""" @@ -577,7 +565,6 @@ class TableExists(google.protobuf.message.Message): global___TableExists = TableExists -@typing_extensions.final class FunctionExists(google.protobuf.message.Message): """See `spark.catalog.functionExists`""" @@ -610,13 +597,11 @@ class FunctionExists(google.protobuf.message.Message): global___FunctionExists = FunctionExists -@typing_extensions.final class CreateExternalTable(google.protobuf.message.Message): """See `spark.catalog.createExternalTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -715,13 +700,11 @@ class CreateExternalTable(google.protobuf.message.Message): global___CreateExternalTable = CreateExternalTable -@typing_extensions.final class CreateTable(google.protobuf.message.Message): """See `spark.catalog.createTable`""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -836,7 +819,6 @@ class CreateTable(google.protobuf.message.Message): global___CreateTable = CreateTable -@typing_extensions.final class DropTempView(google.protobuf.message.Message): """See `spark.catalog.dropTempView`""" @@ -856,7 +838,6 @@ class DropTempView(google.protobuf.message.Message): global___DropTempView = DropTempView -@typing_extensions.final class DropGlobalTempView(google.protobuf.message.Message): """See `spark.catalog.dropGlobalTempView`""" @@ -876,7 +857,6 @@ class DropGlobalTempView(google.protobuf.message.Message): global___DropGlobalTempView = DropGlobalTempView -@typing_extensions.final class RecoverPartitions(google.protobuf.message.Message): """See `spark.catalog.recoverPartitions`""" @@ -896,7 +876,6 @@ class RecoverPartitions(google.protobuf.message.Message): global___RecoverPartitions = RecoverPartitions -@typing_extensions.final class IsCached(google.protobuf.message.Message): """See `spark.catalog.isCached`""" @@ -916,7 +895,6 @@ class IsCached(google.protobuf.message.Message): global___IsCached = IsCached -@typing_extensions.final class CacheTable(google.protobuf.message.Message): """See `spark.catalog.cacheTable`""" @@ -958,7 +936,6 @@ class CacheTable(google.protobuf.message.Message): global___CacheTable = CacheTable -@typing_extensions.final class UncacheTable(google.protobuf.message.Message): """See `spark.catalog.uncacheTable`""" @@ -978,7 +955,6 @@ class UncacheTable(google.protobuf.message.Message): global___UncacheTable = UncacheTable -@typing_extensions.final class ClearCache(google.protobuf.message.Message): """See `spark.catalog.clearCache`""" @@ -990,7 +966,6 @@ class ClearCache(google.protobuf.message.Message): global___ClearCache = ClearCache -@typing_extensions.final class RefreshTable(google.protobuf.message.Message): """See `spark.catalog.refreshTable`""" @@ -1010,7 +985,6 @@ class RefreshTable(google.protobuf.message.Message): global___RefreshTable = RefreshTable -@typing_extensions.final class RefreshByPath(google.protobuf.message.Message): """See `spark.catalog.refreshByPath`""" @@ -1028,7 +1002,6 @@ class RefreshByPath(google.protobuf.message.Message): global___RefreshByPath = RefreshByPath -@typing_extensions.final class CurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.currentCatalog`""" @@ -1040,7 +1013,6 @@ class CurrentCatalog(google.protobuf.message.Message): global___CurrentCatalog = CurrentCatalog -@typing_extensions.final class SetCurrentCatalog(google.protobuf.message.Message): """See `spark.catalog.setCurrentCatalog`""" @@ -1060,7 +1032,6 @@ class SetCurrentCatalog(google.protobuf.message.Message): global___SetCurrentCatalog = SetCurrentCatalog -@typing_extensions.final class ListCatalogs(google.protobuf.message.Message): """See `spark.catalog.listCatalogs`""" diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 5691b16fea31..419a2bd840e3 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -53,7 +53,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Command(google.protobuf.message.Message): """A [[Command]] is an operation that is executed by the server that does not directly consume or produce a relational result. @@ -173,7 +172,6 @@ class Command(google.protobuf.message.Message): global___Command = Command -@typing_extensions.final class SqlCommand(google.protobuf.message.Message): """A SQL Command is used to trigger the eager evaluation of SQL commands in Spark. @@ -185,7 +183,6 @@ class SqlCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -233,7 +230,6 @@ class SqlCommand(google.protobuf.message.Message): global___SqlCommand = SqlCommand -@typing_extensions.final class CreateDataFrameViewCommand(google.protobuf.message.Message): """A command that can create DataFrame global temp view or local temp view.""" @@ -276,7 +272,6 @@ class CreateDataFrameViewCommand(google.protobuf.message.Message): global___CreateDataFrameViewCommand = CreateDataFrameViewCommand -@typing_extensions.final class WriteOperation(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -306,7 +301,6 @@ class WriteOperation(google.protobuf.message.Message): SAVE_MODE_ERROR_IF_EXISTS: WriteOperation.SaveMode.ValueType # 3 SAVE_MODE_IGNORE: WriteOperation.SaveMode.ValueType # 4 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -324,7 +318,6 @@ class WriteOperation(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class SaveTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -367,7 +360,6 @@ class WriteOperation(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class BucketBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -498,7 +490,6 @@ class WriteOperation(google.protobuf.message.Message): global___WriteOperation = WriteOperation -@typing_extensions.final class WriteOperationV2(google.protobuf.message.Message): """As writes are not directly handled during analysis and planning, they are modeled as commands.""" @@ -532,7 +523,6 @@ class WriteOperationV2(google.protobuf.message.Message): MODE_REPLACE: WriteOperationV2.Mode.ValueType # 5 MODE_CREATE_OR_REPLACE: WriteOperationV2.Mode.ValueType # 6 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -550,7 +540,6 @@ class WriteOperationV2(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] ) -> None: ... - @typing_extensions.final class TablePropertiesEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -664,7 +653,6 @@ class WriteOperationV2(google.protobuf.message.Message): global___WriteOperationV2 = WriteOperationV2 -@typing_extensions.final class WriteStreamOperationStart(google.protobuf.message.Message): """Starts write stream operation as streaming query. Query ID and Run ID of the streaming query are returned. @@ -672,7 +660,6 @@ class WriteStreamOperationStart(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -810,7 +797,6 @@ class WriteStreamOperationStart(google.protobuf.message.Message): global___WriteStreamOperationStart = WriteStreamOperationStart -@typing_extensions.final class WriteStreamOperationStartResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -836,7 +822,6 @@ class WriteStreamOperationStartResult(google.protobuf.message.Message): global___WriteStreamOperationStartResult = WriteStreamOperationStartResult -@typing_extensions.final class StreamingQueryInstanceId(google.protobuf.message.Message): """A tuple that uniquely identifies an instance of streaming query run. It consists of `id` that persists across the streaming runs and `run_id` that changes between each run of the @@ -869,13 +854,11 @@ class StreamingQueryInstanceId(google.protobuf.message.Message): global___StreamingQueryInstanceId = StreamingQueryInstanceId -@typing_extensions.final class StreamingQueryCommand(google.protobuf.message.Message): """Commands for a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ExplainCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -977,13 +960,11 @@ class StreamingQueryCommand(google.protobuf.message.Message): global___StreamingQueryCommand = StreamingQueryCommand -@typing_extensions.final class StreamingQueryCommandResult(google.protobuf.message.Message): """Response for commands on a streaming query.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class StatusResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1018,7 +999,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class RecentProgressResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1038,7 +1018,6 @@ class StreamingQueryCommandResult(google.protobuf.message.Message): field_name: typing_extensions.Literal["recent_progress_json", b"recent_progress_json"], ) -> None: ... - @typing_extensions.final class ExplainResult(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/common_pb2.pyi b/python/pyspark/sql/connect/proto/common_pb2.pyi index 03bf86281946..bb7bdeddbfd6 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/common_pb2.pyi @@ -47,7 +47,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class StorageLevel(google.protobuf.message.Message): """StorageLevel for persisting Datasets/Tables.""" diff --git a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi index 4a068a7980b3..1be966ff1e40 100644 --- a/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi +++ b/python/pyspark/sql/connect/proto/example_plugins_pb2.pyi @@ -47,7 +47,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class ExamplePluginRelation(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -72,7 +71,6 @@ class ExamplePluginRelation(google.protobuf.message.Message): global___ExamplePluginRelation = ExamplePluginRelation -@typing_extensions.final class ExamplePluginExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -97,7 +95,6 @@ class ExamplePluginExpression(google.protobuf.message.Message): global___ExamplePluginExpression = ExamplePluginExpression -@typing_extensions.final class ExamplePluginCommand(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index fb2f684546aa..16f84694d2fd 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -51,7 +51,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Expression(google.protobuf.message.Message): """Expression used to refer to fields, functions and similar. This can be used everywhere expressions in SQL appear. @@ -59,13 +58,11 @@ class Expression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Window(google.protobuf.message.Message): """Expression for the OVER clause or WINDOW clause.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class WindowFrame(google.protobuf.message.Message): """The window frame""" @@ -99,7 +96,6 @@ class Expression(google.protobuf.message.Message): All rows having the same 'ORDER BY' ordering are considered as peers. """ - @typing_extensions.final class FrameBoundary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -236,7 +232,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class SortOrder(google.protobuf.message.Message): """SortOrder is used to specify the data ordering, it is normally used in Sort and Window. It is an unevaluable expression and cannot be evaluated, so can not be used in Projection. @@ -311,7 +306,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Cast(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -362,11 +356,9 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["cast_to_type", b"cast_to_type"] ) -> typing_extensions.Literal["type", "type_str"] | None: ... - @typing_extensions.final class Literal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -425,7 +417,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... - @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -449,7 +440,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -653,7 +643,6 @@ class Expression(google.protobuf.message.Message): "array", ] | None: ... - @typing_extensions.final class UnresolvedAttribute(google.protobuf.message.Message): """An unresolved attribute that is not explicitly bound to a specific column, but the column is resolved during analysis by name. @@ -694,7 +683,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... - @typing_extensions.final class UnresolvedFunction(google.protobuf.message.Message): """An unresolved function is not explicitly bound to one explicit function, but the function is resolved during analysis following Sparks name resolution rules. @@ -745,7 +733,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class ExpressionString(google.protobuf.message.Message): """Expression as string.""" @@ -763,7 +750,6 @@ class Expression(google.protobuf.message.Message): self, field_name: typing_extensions.Literal["expression", b"expression"] ) -> None: ... - @typing_extensions.final class UnresolvedStar(google.protobuf.message.Message): """UnresolvedStar is used to expand all the fields of a relation or struct.""" @@ -797,7 +783,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_unparsed_target", b"_unparsed_target"] ) -> typing_extensions.Literal["unparsed_target"] | None: ... - @typing_extensions.final class UnresolvedRegex(google.protobuf.message.Message): """Represents all of the input attributes to a given relational operator, for example in "SELECT `(id)?+.+` FROM ...". @@ -831,7 +816,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_plan_id", b"_plan_id"] ) -> typing_extensions.Literal["plan_id"] | None: ... - @typing_extensions.final class UnresolvedExtractValue(google.protobuf.message.Message): """Extracts a value or values from an Expression""" @@ -864,7 +848,6 @@ class Expression(google.protobuf.message.Message): field_name: typing_extensions.Literal["child", b"child", "extraction", b"extraction"], ) -> None: ... - @typing_extensions.final class UpdateFields(google.protobuf.message.Message): """Add, replace or drop a field of `StructType` expression by name.""" @@ -909,7 +892,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Alias(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -952,7 +934,6 @@ class Expression(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... - @typing_extensions.final class LambdaFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -989,7 +970,6 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class UnresolvedNamedLambdaVariable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1183,7 +1163,6 @@ class Expression(google.protobuf.message.Message): global___Expression = Expression -@typing_extensions.final class CommonInlineUserDefinedFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1256,7 +1235,6 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): global___CommonInlineUserDefinedFunction = CommonInlineUserDefinedFunction -@typing_extensions.final class PythonUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1300,7 +1278,6 @@ class PythonUDF(google.protobuf.message.Message): global___PythonUDF = PythonUDF -@typing_extensions.final class ScalarScalaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1350,7 +1327,6 @@ class ScalarScalaUDF(google.protobuf.message.Message): global___ScalarScalaUDF = ScalarScalaUDF -@typing_extensions.final class JavaUDF(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index 39e2e5237e4f..14552d1f127b 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -53,7 +53,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class Relation(google.protobuf.message.Message): """The main [[Relation]] type. Fundamentally, a relation is a typed container that has exactly one explicit relation type set. @@ -525,7 +524,6 @@ class Relation(google.protobuf.message.Message): global___Relation = Relation -@typing_extensions.final class Unknown(google.protobuf.message.Message): """Used for testing purposes only.""" @@ -537,7 +535,6 @@ class Unknown(google.protobuf.message.Message): global___Unknown = Unknown -@typing_extensions.final class RelationCommon(google.protobuf.message.Message): """Common metadata of all relations.""" @@ -570,13 +567,11 @@ class RelationCommon(google.protobuf.message.Message): global___RelationCommon = RelationCommon -@typing_extensions.final class SQL(google.protobuf.message.Message): """Relation that uses a SQL query to generate the output.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class ArgsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -624,7 +619,6 @@ class SQL(google.protobuf.message.Message): global___SQL = SQL -@typing_extensions.final class Read(google.protobuf.message.Message): """Relation that reads from a file / table or other data source. Does not have additional inputs. @@ -632,11 +626,9 @@ class Read(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class NamedTable(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -676,11 +668,9 @@ class Read(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class DataSource(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -825,7 +815,6 @@ class Read(google.protobuf.message.Message): global___Read = Read -@typing_extensions.final class Project(google.protobuf.message.Message): """Projection of a bag of expressions for a given input relation. @@ -867,7 +856,6 @@ class Project(google.protobuf.message.Message): global___Project = Project -@typing_extensions.final class Filter(google.protobuf.message.Message): """Relation that applies a boolean expression `condition` on each row of `input` to produce the output result. @@ -898,7 +886,6 @@ class Filter(google.protobuf.message.Message): global___Filter = Filter -@typing_extensions.final class Join(google.protobuf.message.Message): """Relation of type [[Join]]. @@ -997,7 +984,6 @@ class Join(google.protobuf.message.Message): global___Join = Join -@typing_extensions.final class SetOperation(google.protobuf.message.Message): """Relation of type [[SetOperation]]""" @@ -1125,7 +1111,6 @@ class SetOperation(google.protobuf.message.Message): global___SetOperation = SetOperation -@typing_extensions.final class Limit(google.protobuf.message.Message): """Relation of type [[Limit]] that is used to `limit` rows from the input relation.""" @@ -1153,7 +1138,6 @@ class Limit(google.protobuf.message.Message): global___Limit = Limit -@typing_extensions.final class Offset(google.protobuf.message.Message): """Relation of type [[Offset]] that is used to read rows staring from the `offset` on the input relation. @@ -1183,7 +1167,6 @@ class Offset(google.protobuf.message.Message): global___Offset = Offset -@typing_extensions.final class Tail(google.protobuf.message.Message): """Relation of type [[Tail]] that is used to fetch `limit` rows from the last of the input relation.""" @@ -1211,7 +1194,6 @@ class Tail(google.protobuf.message.Message): global___Tail = Tail -@typing_extensions.final class Aggregate(google.protobuf.message.Message): """Relation of type [[Aggregate]].""" @@ -1239,7 +1221,6 @@ class Aggregate(google.protobuf.message.Message): GROUP_TYPE_CUBE: Aggregate.GroupType.ValueType # 3 GROUP_TYPE_PIVOT: Aggregate.GroupType.ValueType # 4 - @typing_extensions.final class Pivot(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -1338,7 +1319,6 @@ class Aggregate(google.protobuf.message.Message): global___Aggregate = Aggregate -@typing_extensions.final class Sort(google.protobuf.message.Message): """Relation of type [[Sort]].""" @@ -1394,7 +1374,6 @@ class Sort(google.protobuf.message.Message): global___Sort = Sort -@typing_extensions.final class Drop(google.protobuf.message.Message): """Drop specified columns.""" @@ -1438,7 +1417,6 @@ class Drop(google.protobuf.message.Message): global___Drop = Drop -@typing_extensions.final class Deduplicate(google.protobuf.message.Message): """Relation of type [[Deduplicate]] which have duplicate rows removed, could consider either only the subset of columns or all the columns. @@ -1503,7 +1481,6 @@ class Deduplicate(google.protobuf.message.Message): global___Deduplicate = Deduplicate -@typing_extensions.final class LocalRelation(google.protobuf.message.Message): """A relation that does not need to be qualified by name.""" @@ -1551,7 +1528,6 @@ class LocalRelation(google.protobuf.message.Message): global___LocalRelation = LocalRelation -@typing_extensions.final class Sample(google.protobuf.message.Message): """Relation of type [[Sample]] that samples a fraction of the dataset.""" @@ -1636,7 +1612,6 @@ class Sample(google.protobuf.message.Message): global___Sample = Sample -@typing_extensions.final class Range(google.protobuf.message.Message): """Relation of type [[Range]] that generates a sequence of integers.""" @@ -1705,7 +1680,6 @@ class Range(google.protobuf.message.Message): global___Range = Range -@typing_extensions.final class SubqueryAlias(google.protobuf.message.Message): """Relation alias.""" @@ -1743,7 +1717,6 @@ class SubqueryAlias(google.protobuf.message.Message): global___SubqueryAlias = SubqueryAlias -@typing_extensions.final class Repartition(google.protobuf.message.Message): """Relation repartition.""" @@ -1791,7 +1764,6 @@ class Repartition(google.protobuf.message.Message): global___Repartition = Repartition -@typing_extensions.final class ShowString(google.protobuf.message.Message): """Compose the string representing rows for output. It will invoke 'Dataset.showString' to compute the results. @@ -1841,7 +1813,6 @@ class ShowString(google.protobuf.message.Message): global___ShowString = ShowString -@typing_extensions.final class StatSummary(google.protobuf.message.Message): """Computes specified statistics for numeric and string columns. It will invoke 'Dataset.summary' (same as 'StatFunctions.summary') @@ -1889,7 +1860,6 @@ class StatSummary(google.protobuf.message.Message): global___StatSummary = StatSummary -@typing_extensions.final class StatDescribe(google.protobuf.message.Message): """Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or @@ -1923,7 +1893,6 @@ class StatDescribe(google.protobuf.message.Message): global___StatDescribe = StatDescribe -@typing_extensions.final class StatCrosstab(google.protobuf.message.Message): """Computes a pair-wise frequency table of the given columns. Also known as a contingency table. It will invoke 'Dataset.stat.crosstab' (same as 'StatFunctions.crossTabulate') @@ -1965,7 +1934,6 @@ class StatCrosstab(google.protobuf.message.Message): global___StatCrosstab = StatCrosstab -@typing_extensions.final class StatCov(google.protobuf.message.Message): """Calculate the sample covariance of two numerical columns of a DataFrame. It will invoke 'Dataset.stat.cov' (same as 'StatFunctions.calculateCov') to compute the results. @@ -2000,7 +1968,6 @@ class StatCov(google.protobuf.message.Message): global___StatCov = StatCov -@typing_extensions.final class StatCorr(google.protobuf.message.Message): """Calculates the correlation of two columns of a DataFrame. Currently only supports the Pearson Correlation Coefficient. It will invoke 'Dataset.stat.corr' (same as @@ -2060,7 +2027,6 @@ class StatCorr(google.protobuf.message.Message): global___StatCorr = StatCorr -@typing_extensions.final class StatApproxQuantile(google.protobuf.message.Message): """Calculates the approximate quantiles of numerical columns of a DataFrame. It will invoke 'Dataset.stat.approxQuantile' (same as 'StatFunctions.approxQuantile') @@ -2123,7 +2089,6 @@ class StatApproxQuantile(google.protobuf.message.Message): global___StatApproxQuantile = StatApproxQuantile -@typing_extensions.final class StatFreqItems(google.protobuf.message.Message): """Finding frequent items for columns, possibly with false positives. It will invoke 'Dataset.stat.freqItems' (same as 'StatFunctions.freqItems') @@ -2172,7 +2137,6 @@ class StatFreqItems(google.protobuf.message.Message): global___StatFreqItems = StatFreqItems -@typing_extensions.final class StatSampleBy(google.protobuf.message.Message): """Returns a stratified sample without replacement based on the fraction given on each stratum. @@ -2182,7 +2146,6 @@ class StatSampleBy(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Fraction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2264,7 +2227,6 @@ class StatSampleBy(google.protobuf.message.Message): global___StatSampleBy = StatSampleBy -@typing_extensions.final class NAFill(google.protobuf.message.Message): """Replaces null values. It will invoke 'Dataset.na.fill' (same as 'DataFrameNaFunctions.fill') to compute the results. @@ -2323,7 +2285,6 @@ class NAFill(google.protobuf.message.Message): global___NAFill = NAFill -@typing_extensions.final class NADrop(google.protobuf.message.Message): """Drop rows containing null values. It will invoke 'Dataset.na.drop' (same as 'DataFrameNaFunctions.drop') to compute the results. @@ -2392,7 +2353,6 @@ class NADrop(google.protobuf.message.Message): global___NADrop = NADrop -@typing_extensions.final class NAReplace(google.protobuf.message.Message): """Replaces old values with the corresponding values. It will invoke 'Dataset.na.replace' (same as 'DataFrameNaFunctions.replace') @@ -2401,7 +2361,6 @@ class NAReplace(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Replacement(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2478,7 +2437,6 @@ class NAReplace(google.protobuf.message.Message): global___NAReplace = NAReplace -@typing_extensions.final class ToDF(google.protobuf.message.Message): """Rename columns on the input relation by the same length of names.""" @@ -2514,13 +2472,11 @@ class ToDF(google.protobuf.message.Message): global___ToDF = ToDF -@typing_extensions.final class WithColumnsRenamed(google.protobuf.message.Message): """Rename columns on the input relation by a map with name to name mapping.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class RenameColumnsMapEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2572,7 +2528,6 @@ class WithColumnsRenamed(google.protobuf.message.Message): global___WithColumnsRenamed = WithColumnsRenamed -@typing_extensions.final class WithColumns(google.protobuf.message.Message): """Adding columns or replacing the existing columns that have the same names.""" @@ -2617,7 +2572,6 @@ class WithColumns(google.protobuf.message.Message): global___WithColumns = WithColumns -@typing_extensions.final class WithWatermark(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2650,7 +2604,6 @@ class WithWatermark(google.protobuf.message.Message): global___WithWatermark = WithWatermark -@typing_extensions.final class Hint(google.protobuf.message.Message): """Specify a hint over a relation. Hint should have a name and optional parameters.""" @@ -2696,13 +2649,11 @@ class Hint(google.protobuf.message.Message): global___Hint = Hint -@typing_extensions.final class Unpivot(google.protobuf.message.Message): """Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set.""" DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Values(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2784,7 +2735,6 @@ class Unpivot(google.protobuf.message.Message): global___Unpivot = Unpivot -@typing_extensions.final class ToSchema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2814,7 +2764,6 @@ class ToSchema(google.protobuf.message.Message): global___ToSchema = ToSchema -@typing_extensions.final class RepartitionByExpression(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2873,7 +2822,6 @@ class RepartitionByExpression(google.protobuf.message.Message): global___RepartitionByExpression = RepartitionByExpression -@typing_extensions.final class MapPartitions(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2928,7 +2876,6 @@ class MapPartitions(google.protobuf.message.Message): global___MapPartitions = MapPartitions -@typing_extensions.final class GroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -2971,7 +2918,6 @@ class GroupMap(google.protobuf.message.Message): global___GroupMap = GroupMap -@typing_extensions.final class CoGroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3043,7 +2989,6 @@ class CoGroupMap(google.protobuf.message.Message): global___CoGroupMap = CoGroupMap -@typing_extensions.final class ApplyInPandasWithState(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3115,7 +3060,6 @@ class ApplyInPandasWithState(google.protobuf.message.Message): global___ApplyInPandasWithState = ApplyInPandasWithState -@typing_extensions.final class CollectMetrics(google.protobuf.message.Message): """Collect arbitrary (named) metrics from a dataset.""" @@ -3156,7 +3100,6 @@ class CollectMetrics(google.protobuf.message.Message): global___CollectMetrics = CollectMetrics -@typing_extensions.final class Parse(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -3178,7 +3121,6 @@ class Parse(google.protobuf.message.Message): PARSE_FORMAT_CSV: Parse.ParseFormat.ValueType # 1 PARSE_FORMAT_JSON: Parse.ParseFormat.ValueType # 2 - @typing_extensions.final class OptionsEntry(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index c4b17db1abf0..956701b4c366 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -48,7 +48,6 @@ else: DESCRIPTOR: google.protobuf.descriptor.FileDescriptor -@typing_extensions.final class DataType(google.protobuf.message.Message): """This message describes the logical [[DataType]] of something. It does not carry the value itself but only describes it. @@ -56,7 +55,6 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - @typing_extensions.final class Boolean(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -74,7 +72,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Byte(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -92,7 +89,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Short(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -110,7 +106,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Integer(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -128,7 +123,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Long(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -146,7 +140,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Float(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -164,7 +157,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Double(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -182,7 +174,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class String(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -200,7 +191,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Binary(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -218,7 +208,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class NULL(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -236,7 +225,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Timestamp(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -254,7 +242,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Date(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -272,7 +259,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class TimestampNTZ(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -290,7 +276,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class CalendarInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -308,7 +293,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class YearMonthInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -362,7 +346,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... - @typing_extensions.final class DayTimeInterval(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -416,7 +399,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_start_field", b"_start_field"] ) -> typing_extensions.Literal["start_field"] | None: ... - @typing_extensions.final class Char(google.protobuf.message.Message): """Start compound types.""" @@ -439,7 +421,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class VarChar(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -460,7 +441,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Decimal(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -514,7 +494,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_scale", b"_scale"] ) -> typing_extensions.Literal["scale"] | None: ... - @typing_extensions.final class StructField(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -560,7 +539,6 @@ class DataType(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_metadata", b"_metadata"] ) -> typing_extensions.Literal["metadata"] | None: ... - @typing_extensions.final class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -586,7 +564,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Array(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -619,7 +596,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class Map(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -661,7 +637,6 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... - @typing_extensions.final class UDT(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor @@ -741,7 +716,6 @@ class DataType(google.protobuf.message.Message): ], ) -> typing_extensions.Literal["serialized_python_class"] | None: ... - @typing_extensions.final class Unparsed(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From e2024249345e7d966cbfcb5b843d3dcfade60d24 Mon Sep 17 00:00:00 2001 From: pengzhon-db Date: Sun, 16 Apr 2023 13:29:05 -0700 Subject: [PATCH 12/12] revert change --- dev/tox.ini | 1 - 1 file changed, 1 deletion(-) diff --git a/dev/tox.ini b/dev/tox.ini index 2751c5521cc6..c6edee272add 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -57,5 +57,4 @@ exclude = python/pyspark/java_gateway.pyi, dev/ansible-for-test-node/*, python/pyspark/sql/connect/proto/*, - */venv/* max-line-length = 100