From ac9a8839a7ecdf579d0350bbcf90f2f6cd1f03ad Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 17 Jan 2025 14:59:28 -0800 Subject: [PATCH 01/20] draft save --- dev/sparktestsupport/modules.py | 1 + ...test_parity_pandas_transform_with_state.py | 40 +++++++++++++++++++ .../protobuf/spark/connect/relations.proto | 30 ++++++++++++++ 3 files changed, 71 insertions(+) create mode 100644 python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index dd7c387cd5f1e..6b935c4378764 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1096,6 +1096,7 @@ def __hash__(self): "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_scalar", "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_grouped_agg", "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_window", + "pyspark.sql.tests.connect.pandas.test_parity_pandas_transform_with_state" ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py new file mode 100644 index 0000000000000..cd78bf209b5e2 --- /dev/null +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -0,0 +1,40 @@ +# +# 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_transform_with_state import ( + TransformWithStateInPandasTestsMixin, +) +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class TransformWithStateInPandasTests( + TransformWithStateInPandasTestsMixin, ReusedConnectTestCase +): + pass + + +if __name__ == "__main__": + from pyspark.sql.tests.connect.pandas.test_parity_pandas_transform_with_state import * # noqa: F401,E501 + + 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/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index 58ce057859787..886d66e1e6e15 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -80,6 +80,7 @@ message Relation { Transpose transpose = 42; UnresolvedTableValuedFunction unresolved_table_valued_function = 43; LateralJoin lateral_join = 44; + TransformWithStateInPandas transform_with_state_in_pandas = 45; // NA functions NAFill fill_na = 90; @@ -1079,6 +1080,35 @@ message ApplyInPandasWithState { string timeout_conf = 7; } +message TransformWithStateInPandas { + // (Required) Input relation for transformWithStateInPandas. + 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) The output mode of the function. + string output_mode = 5; + + // (Required) Time mode for transformWithStateInPandas + string time_mode = 6; + + // (Optional) Input relation for initial State. + Relation initial_input = 7; + + // (Optional) Expressions for grouping keys of the initial state input relation. + repeated Expression initial_grouping_expressions = 8; + + // (Optional) Event time column name + optional string event_time_col_name = 9; +} + message CommonInlineUserDefinedTableFunction { // (Required) Name of the user-defined table function. string function_name = 1; From f667498b5c1fae1942b6217dbed23b6e52d92c6a Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 23 Jan 2025 16:56:34 -0800 Subject: [PATCH 02/20] basic value state is working --- python/pyspark/sql/connect/group.py | 160 +++++++++ python/pyspark/sql/connect/plan.py | 40 +++ .../sql/connect/proto/relations_pb2.py | 334 +++++++++--------- .../sql/connect/proto/relations_pb2.pyi | 121 +++++++ ...test_parity_pandas_transform_with_state.py | 2 +- .../test_pandas_transform_with_state.py | 46 +-- .../protobuf/spark/connect/relations.proto | 4 +- .../connect/planner/SparkConnectPlanner.scala | 21 ++ 8 files changed, 532 insertions(+), 196 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 11adc8850fec1..d60a0e35c7af3 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -41,6 +41,19 @@ from pyspark.sql.column import Column from pyspark.sql.connect.functions import builtin as F from pyspark.errors import PySparkNotImplementedError, PySparkTypeError +from pyspark.sql.streaming.stateful_processor_api_client import ( + StatefulProcessorApiClient, + StatefulProcessorHandleState, +) +from pyspark.sql.streaming.stateful_processor import ( + ExpiredTimerInfo, + StatefulProcessor, + StatefulProcessorHandle, + TimerValues, +) +from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle +from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode + if TYPE_CHECKING: from pyspark.sql.connect._typing import ( @@ -361,6 +374,153 @@ def applyInPandasWithState( applyInPandasWithState.__doc__ = PySparkGroupedData.applyInPandasWithState.__doc__ + def transformWithStateInPandas( + self, + statefulProcessor: StatefulProcessor, + outputStructType: Union[StructType, str], + outputMode: str, + timeMode: str, + initialState: Optional["GroupedData"] = None, + eventTimeColumnName: str = "", + ) -> "DataFrame": + from pyspark.sql.connect.udf import UserDefinedFunction + from pyspark.sql.connect.dataframe import DataFrame + import itertools + from typing import Any, Iterator + + def handle_pre_init( + statefulProcessorApiClient: StatefulProcessorApiClient, + ) -> Iterator["PandasDataFrameLike"]: + # Driver handle is different from the handle used on executors; + # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which + # will only be used for handling init() and get the state schema on the driver. + driver_handle = StatefulProcessorHandle(statefulProcessorApiClient) + statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.PRE_INIT) + statefulProcessor.init(driver_handle) + + # This method is used for the driver-side stateful processor after we have collected + # all the necessary schemas. This instance of the DriverStatefulProcessorHandleImpl + # won't be used again on JVM. + statefulProcessor.close() + + # return a dummy results, no return value is needed for pre init + return iter([]) + + def handle_data_rows( + statefulProcessorApiClient: StatefulProcessorApiClient, + key: Any, + inputRows: Optional[Iterator["PandasDataFrameLike"]] = None, + ) -> Iterator["PandasDataFrameLike"]: + statefulProcessorApiClient.set_implicit_key(key) + + batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( + timeMode + ) + + # process with data rows + if inputRows is not None: + data_iter = statefulProcessor.handleInputRows( + key, inputRows, TimerValues(batch_timestamp, watermark_timestamp) + ) + return data_iter + else: + return iter([]) + + def handle_expired_timers( + statefulProcessorApiClient: StatefulProcessorApiClient, + ) -> Iterator["PandasDataFrameLike"]: + batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( + timeMode + ) + + if timeMode.lower() == "processingtime": + expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( + batch_timestamp + ) + elif timeMode.lower() == "eventtime": + expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( + watermark_timestamp + ) + else: + expiry_list_iter = iter([[]]) + + # process with expiry timers, only timer related rows will be emitted + for expiry_list in expiry_list_iter: + for key_obj, expiry_timestamp in expiry_list: + statefulProcessorApiClient.set_implicit_key(key_obj) + for pd in statefulProcessor.handleExpiredTimer( + key=key_obj, + timer_values=TimerValues(batch_timestamp, watermark_timestamp), + expired_timer_info=ExpiredTimerInfo(expiry_timestamp), + ): + yield pd + statefulProcessorApiClient.delete_timer(expiry_timestamp) + + def transformWithStateUDF( + statefulProcessorApiClient: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + inputRows: Iterator["PandasDataFrameLike"], + ) -> Iterator["PandasDataFrameLike"]: + if mode == TransformWithStateInPandasFuncMode.PRE_INIT: + return handle_pre_init(statefulProcessorApiClient) + + handle = StatefulProcessorHandle(statefulProcessorApiClient) + + if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: + statefulProcessor.init(handle) + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.INITIALIZED + ) + + if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.DATA_PROCESSED + ) + result = handle_expired_timers(statefulProcessorApiClient) + return result + elif mode == TransformWithStateInPandasFuncMode.COMPLETE: + statefulProcessorApiClient.set_handle_state( + StatefulProcessorHandleState.TIMER_PROCESSED + ) + statefulProcessorApiClient.remove_implicit_key() + statefulProcessor.close() + statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) + return iter([]) + else: + # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA + result = handle_data_rows(statefulProcessorApiClient, key, inputRows) + return result + + # TODO add initial state + udf_obj = UserDefinedFunction( + transformWithStateUDF, + returnType=outputStructType, + evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, + ) + + # TODO figure out if we need to handle for string type + output_schema: str = ( + outputStructType.json() + if isinstance(outputStructType, StructType) + else outputStructType + ) + + return DataFrame( + plan.TransformWithStateInPandas( + child=self._df._plan, + grouping_cols=self._grouping_cols, + function=udf_obj, + output_schema=output_schema, + output_mode=outputMode, + time_mode=timeMode, + cols=self._df.columns, + ), + session=self._df._session, + ) + + applyInPandasWithState.__doc__ = PySparkGroupedData.applyInPandasWithState.__doc__ + def applyInArrow( self, func: "ArrowGroupedMapFunction", schema: Union[StructType, str] ) -> "DataFrame": diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 02b60381ab939..26b6f85bd4795 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -59,6 +59,7 @@ PySparkValueError, PySparkPicklingError, ) +from pyspark.sql.streaming.stateful_processor import StatefulProcessor if TYPE_CHECKING: from pyspark.sql.connect.client import SparkConnectClient @@ -2546,6 +2547,45 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return self._with_relations(plan, session) +class TransformWithStateInPandas(LogicalPlan): + """Logical plan object for a applyInPandasWithState.""" + + def __init__( + self, + child: Optional["LogicalPlan"], + grouping_cols: Sequence[Column], + function: "UserDefinedFunction", + output_schema: str, + output_mode: str, + time_mode: str, + cols: List[str], + ): + assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) + + super().__init__(child, self._collect_references(grouping_cols)) + self._grouping_cols = grouping_cols + self._output_schema = output_schema + self._output_mode = output_mode + self._time_mode = time_mode + self._function = function._build_common_inline_user_defined_function(*cols) + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.transform_with_state_in_pandas.input.CopyFrom(self._child.plan(session)) + plan.transform_with_state_in_pandas.grouping_expressions.extend( + [c.to_plan(session) for c in self._grouping_cols] + ) + + # this is to deserialize stateful processor + plan.transform_with_state_in_pandas.transform_with_state_udf.CopyFrom( + self._function.to_plan_udf(session)) + plan.transform_with_state_in_pandas.output_schema = self._output_schema + plan.transform_with_state_in_pandas.output_mode = self._output_mode + plan.transform_with_state_in_pandas.time_mode = self._time_mode + return self._with_relations(plan, session) + + class PythonUDTF: """Represents a Python user-defined table function.""" diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 422addcf3dfb7..94b0717b190ee 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x8d\x1e\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12o\n\x1etransform_with_state_in_pandas\x18- \x01(\x0b\x32).spark.connect.TransformWithStateInPandasH\x00R\x1atransformWithStateInPandas\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xcc\x04\n\x1aTransformWithStateInPandas\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\x12g\n\x18transform_with_state_udf\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x15transformWithStateUdf\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12\x1f\n\x0boutput_mode\x18\x05 \x01(\tR\noutputMode\x12\x1b\n\ttime_mode\x18\x06 \x01(\tR\x08timeMode\x12<\n\rinitial_input\x18\x07 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x08 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12\x32\n\x13\x65vent_time_col_name\x18\t \x01(\tH\x00R\x10\x65ventTimeColName\x88\x01\x01\x42\x16\n\x14_event_time_col_name"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -79,169 +79,171 @@ _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_RELATION"]._serialized_start = 224 - _globals["_RELATION"]._serialized_end = 3964 - _globals["_MLRELATION"]._serialized_start = 3967 - _globals["_MLRELATION"]._serialized_end = 4343 - _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4097 - _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4332 - _globals["_FETCH"]._serialized_start = 4346 - _globals["_FETCH"]._serialized_end = 4677 - _globals["_FETCH_METHOD"]._serialized_start = 4462 - _globals["_FETCH_METHOD"]._serialized_end = 4677 - _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4550 - _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4677 - _globals["_UNKNOWN"]._serialized_start = 4679 - _globals["_UNKNOWN"]._serialized_end = 4688 - _globals["_RELATIONCOMMON"]._serialized_start = 4691 - _globals["_RELATIONCOMMON"]._serialized_end = 4833 - _globals["_SQL"]._serialized_start = 4836 - _globals["_SQL"]._serialized_end = 5314 - _globals["_SQL_ARGSENTRY"]._serialized_start = 5130 - _globals["_SQL_ARGSENTRY"]._serialized_end = 5220 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5222 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5314 - _globals["_WITHRELATIONS"]._serialized_start = 5316 - _globals["_WITHRELATIONS"]._serialized_end = 5433 - _globals["_READ"]._serialized_start = 5436 - _globals["_READ"]._serialized_end = 6099 - _globals["_READ_NAMEDTABLE"]._serialized_start = 5614 - _globals["_READ_NAMEDTABLE"]._serialized_end = 5806 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5748 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5806 - _globals["_READ_DATASOURCE"]._serialized_start = 5809 - _globals["_READ_DATASOURCE"]._serialized_end = 6086 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5748 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5806 - _globals["_PROJECT"]._serialized_start = 6101 - _globals["_PROJECT"]._serialized_end = 6218 - _globals["_FILTER"]._serialized_start = 6220 - _globals["_FILTER"]._serialized_end = 6332 - _globals["_JOIN"]._serialized_start = 6335 - _globals["_JOIN"]._serialized_end = 6996 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6674 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6766 - _globals["_JOIN_JOINTYPE"]._serialized_start = 6769 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6977 - _globals["_SETOPERATION"]._serialized_start = 6999 - _globals["_SETOPERATION"]._serialized_end = 7478 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7315 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7429 - _globals["_LIMIT"]._serialized_start = 7480 - _globals["_LIMIT"]._serialized_end = 7556 - _globals["_OFFSET"]._serialized_start = 7558 - _globals["_OFFSET"]._serialized_end = 7637 - _globals["_TAIL"]._serialized_start = 7639 - _globals["_TAIL"]._serialized_end = 7714 - _globals["_AGGREGATE"]._serialized_start = 7717 - _globals["_AGGREGATE"]._serialized_end = 8483 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 8132 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 8243 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8245 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8321 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8324 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8483 - _globals["_SORT"]._serialized_start = 8486 - _globals["_SORT"]._serialized_end = 8646 - _globals["_DROP"]._serialized_start = 8649 - _globals["_DROP"]._serialized_end = 8790 - _globals["_DEDUPLICATE"]._serialized_start = 8793 - _globals["_DEDUPLICATE"]._serialized_end = 9033 - _globals["_LOCALRELATION"]._serialized_start = 9035 - _globals["_LOCALRELATION"]._serialized_end = 9124 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 9126 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 9198 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 9200 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 9255 - _globals["_SAMPLE"]._serialized_start = 9258 - _globals["_SAMPLE"]._serialized_end = 9531 - _globals["_RANGE"]._serialized_start = 9534 - _globals["_RANGE"]._serialized_end = 9679 - _globals["_SUBQUERYALIAS"]._serialized_start = 9681 - _globals["_SUBQUERYALIAS"]._serialized_end = 9795 - _globals["_REPARTITION"]._serialized_start = 9798 - _globals["_REPARTITION"]._serialized_end = 9940 - _globals["_SHOWSTRING"]._serialized_start = 9943 - _globals["_SHOWSTRING"]._serialized_end = 10085 - _globals["_HTMLSTRING"]._serialized_start = 10087 - _globals["_HTMLSTRING"]._serialized_end = 10201 - _globals["_STATSUMMARY"]._serialized_start = 10203 - _globals["_STATSUMMARY"]._serialized_end = 10295 - _globals["_STATDESCRIBE"]._serialized_start = 10297 - _globals["_STATDESCRIBE"]._serialized_end = 10378 - _globals["_STATCROSSTAB"]._serialized_start = 10380 - _globals["_STATCROSSTAB"]._serialized_end = 10481 - _globals["_STATCOV"]._serialized_start = 10483 - _globals["_STATCOV"]._serialized_end = 10579 - _globals["_STATCORR"]._serialized_start = 10582 - _globals["_STATCORR"]._serialized_end = 10719 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 10722 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10886 - _globals["_STATFREQITEMS"]._serialized_start = 10888 - _globals["_STATFREQITEMS"]._serialized_end = 11013 - _globals["_STATSAMPLEBY"]._serialized_start = 11016 - _globals["_STATSAMPLEBY"]._serialized_end = 11325 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11217 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11316 - _globals["_NAFILL"]._serialized_start = 11328 - _globals["_NAFILL"]._serialized_end = 11462 - _globals["_NADROP"]._serialized_start = 11465 - _globals["_NADROP"]._serialized_end = 11599 - _globals["_NAREPLACE"]._serialized_start = 11602 - _globals["_NAREPLACE"]._serialized_end = 11898 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11757 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11898 - _globals["_TODF"]._serialized_start = 11900 - _globals["_TODF"]._serialized_end = 11988 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11991 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12373 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12235 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12302 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12304 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12373 - _globals["_WITHCOLUMNS"]._serialized_start = 12375 - _globals["_WITHCOLUMNS"]._serialized_end = 12494 - _globals["_WITHWATERMARK"]._serialized_start = 12497 - _globals["_WITHWATERMARK"]._serialized_end = 12631 - _globals["_HINT"]._serialized_start = 12634 - _globals["_HINT"]._serialized_end = 12766 - _globals["_UNPIVOT"]._serialized_start = 12769 - _globals["_UNPIVOT"]._serialized_end = 13096 - _globals["_UNPIVOT_VALUES"]._serialized_start = 13026 - _globals["_UNPIVOT_VALUES"]._serialized_end = 13085 - _globals["_TRANSPOSE"]._serialized_start = 13098 - _globals["_TRANSPOSE"]._serialized_end = 13220 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13222 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13347 - _globals["_TOSCHEMA"]._serialized_start = 13349 - _globals["_TOSCHEMA"]._serialized_end = 13455 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13458 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13661 - _globals["_MAPPARTITIONS"]._serialized_start = 13664 - _globals["_MAPPARTITIONS"]._serialized_end = 13896 - _globals["_GROUPMAP"]._serialized_start = 13899 - _globals["_GROUPMAP"]._serialized_end = 14616 - _globals["_COGROUPMAP"]._serialized_start = 14619 - _globals["_COGROUPMAP"]._serialized_end = 15145 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15148 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15505 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15508 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 15752 - _globals["_PYTHONUDTF"]._serialized_start = 15755 - _globals["_PYTHONUDTF"]._serialized_end = 15932 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15935 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16086 - _globals["_PYTHONDATASOURCE"]._serialized_start = 16088 - _globals["_PYTHONDATASOURCE"]._serialized_end = 16163 - _globals["_COLLECTMETRICS"]._serialized_start = 16166 - _globals["_COLLECTMETRICS"]._serialized_end = 16302 - _globals["_PARSE"]._serialized_start = 16305 - _globals["_PARSE"]._serialized_end = 16693 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5748 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5806 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16594 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 16682 - _globals["_ASOFJOIN"]._serialized_start = 16696 - _globals["_ASOFJOIN"]._serialized_end = 17171 - _globals["_LATERALJOIN"]._serialized_start = 17174 - _globals["_LATERALJOIN"]._serialized_end = 17404 + _globals["_RELATION"]._serialized_end = 4077 + _globals["_MLRELATION"]._serialized_start = 4080 + _globals["_MLRELATION"]._serialized_end = 4456 + _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4210 + _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4445 + _globals["_FETCH"]._serialized_start = 4459 + _globals["_FETCH"]._serialized_end = 4790 + _globals["_FETCH_METHOD"]._serialized_start = 4575 + _globals["_FETCH_METHOD"]._serialized_end = 4790 + _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4663 + _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4790 + _globals["_UNKNOWN"]._serialized_start = 4792 + _globals["_UNKNOWN"]._serialized_end = 4801 + _globals["_RELATIONCOMMON"]._serialized_start = 4804 + _globals["_RELATIONCOMMON"]._serialized_end = 4946 + _globals["_SQL"]._serialized_start = 4949 + _globals["_SQL"]._serialized_end = 5427 + _globals["_SQL_ARGSENTRY"]._serialized_start = 5243 + _globals["_SQL_ARGSENTRY"]._serialized_end = 5333 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5335 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5427 + _globals["_WITHRELATIONS"]._serialized_start = 5429 + _globals["_WITHRELATIONS"]._serialized_end = 5546 + _globals["_READ"]._serialized_start = 5549 + _globals["_READ"]._serialized_end = 6212 + _globals["_READ_NAMEDTABLE"]._serialized_start = 5727 + _globals["_READ_NAMEDTABLE"]._serialized_end = 5919 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5861 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5919 + _globals["_READ_DATASOURCE"]._serialized_start = 5922 + _globals["_READ_DATASOURCE"]._serialized_end = 6199 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5861 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5919 + _globals["_PROJECT"]._serialized_start = 6214 + _globals["_PROJECT"]._serialized_end = 6331 + _globals["_FILTER"]._serialized_start = 6333 + _globals["_FILTER"]._serialized_end = 6445 + _globals["_JOIN"]._serialized_start = 6448 + _globals["_JOIN"]._serialized_end = 7109 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6787 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6879 + _globals["_JOIN_JOINTYPE"]._serialized_start = 6882 + _globals["_JOIN_JOINTYPE"]._serialized_end = 7090 + _globals["_SETOPERATION"]._serialized_start = 7112 + _globals["_SETOPERATION"]._serialized_end = 7591 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7428 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7542 + _globals["_LIMIT"]._serialized_start = 7593 + _globals["_LIMIT"]._serialized_end = 7669 + _globals["_OFFSET"]._serialized_start = 7671 + _globals["_OFFSET"]._serialized_end = 7750 + _globals["_TAIL"]._serialized_start = 7752 + _globals["_TAIL"]._serialized_end = 7827 + _globals["_AGGREGATE"]._serialized_start = 7830 + _globals["_AGGREGATE"]._serialized_end = 8596 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 8245 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 8356 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8358 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8434 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8437 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8596 + _globals["_SORT"]._serialized_start = 8599 + _globals["_SORT"]._serialized_end = 8759 + _globals["_DROP"]._serialized_start = 8762 + _globals["_DROP"]._serialized_end = 8903 + _globals["_DEDUPLICATE"]._serialized_start = 8906 + _globals["_DEDUPLICATE"]._serialized_end = 9146 + _globals["_LOCALRELATION"]._serialized_start = 9148 + _globals["_LOCALRELATION"]._serialized_end = 9237 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 9239 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 9311 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 9313 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 9368 + _globals["_SAMPLE"]._serialized_start = 9371 + _globals["_SAMPLE"]._serialized_end = 9644 + _globals["_RANGE"]._serialized_start = 9647 + _globals["_RANGE"]._serialized_end = 9792 + _globals["_SUBQUERYALIAS"]._serialized_start = 9794 + _globals["_SUBQUERYALIAS"]._serialized_end = 9908 + _globals["_REPARTITION"]._serialized_start = 9911 + _globals["_REPARTITION"]._serialized_end = 10053 + _globals["_SHOWSTRING"]._serialized_start = 10056 + _globals["_SHOWSTRING"]._serialized_end = 10198 + _globals["_HTMLSTRING"]._serialized_start = 10200 + _globals["_HTMLSTRING"]._serialized_end = 10314 + _globals["_STATSUMMARY"]._serialized_start = 10316 + _globals["_STATSUMMARY"]._serialized_end = 10408 + _globals["_STATDESCRIBE"]._serialized_start = 10410 + _globals["_STATDESCRIBE"]._serialized_end = 10491 + _globals["_STATCROSSTAB"]._serialized_start = 10493 + _globals["_STATCROSSTAB"]._serialized_end = 10594 + _globals["_STATCOV"]._serialized_start = 10596 + _globals["_STATCOV"]._serialized_end = 10692 + _globals["_STATCORR"]._serialized_start = 10695 + _globals["_STATCORR"]._serialized_end = 10832 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 10835 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10999 + _globals["_STATFREQITEMS"]._serialized_start = 11001 + _globals["_STATFREQITEMS"]._serialized_end = 11126 + _globals["_STATSAMPLEBY"]._serialized_start = 11129 + _globals["_STATSAMPLEBY"]._serialized_end = 11438 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11330 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11429 + _globals["_NAFILL"]._serialized_start = 11441 + _globals["_NAFILL"]._serialized_end = 11575 + _globals["_NADROP"]._serialized_start = 11578 + _globals["_NADROP"]._serialized_end = 11712 + _globals["_NAREPLACE"]._serialized_start = 11715 + _globals["_NAREPLACE"]._serialized_end = 12011 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11870 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 12011 + _globals["_TODF"]._serialized_start = 12013 + _globals["_TODF"]._serialized_end = 12101 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 12104 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12486 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12348 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12415 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12417 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12486 + _globals["_WITHCOLUMNS"]._serialized_start = 12488 + _globals["_WITHCOLUMNS"]._serialized_end = 12607 + _globals["_WITHWATERMARK"]._serialized_start = 12610 + _globals["_WITHWATERMARK"]._serialized_end = 12744 + _globals["_HINT"]._serialized_start = 12747 + _globals["_HINT"]._serialized_end = 12879 + _globals["_UNPIVOT"]._serialized_start = 12882 + _globals["_UNPIVOT"]._serialized_end = 13209 + _globals["_UNPIVOT_VALUES"]._serialized_start = 13139 + _globals["_UNPIVOT_VALUES"]._serialized_end = 13198 + _globals["_TRANSPOSE"]._serialized_start = 13211 + _globals["_TRANSPOSE"]._serialized_end = 13333 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13335 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13460 + _globals["_TOSCHEMA"]._serialized_start = 13462 + _globals["_TOSCHEMA"]._serialized_end = 13568 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13571 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13774 + _globals["_MAPPARTITIONS"]._serialized_start = 13777 + _globals["_MAPPARTITIONS"]._serialized_end = 14009 + _globals["_GROUPMAP"]._serialized_start = 14012 + _globals["_GROUPMAP"]._serialized_end = 14729 + _globals["_COGROUPMAP"]._serialized_start = 14732 + _globals["_COGROUPMAP"]._serialized_end = 15258 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15261 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15618 + _globals["_TRANSFORMWITHSTATEINPANDAS"]._serialized_start = 15621 + _globals["_TRANSFORMWITHSTATEINPANDAS"]._serialized_end = 16209 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 16212 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16456 + _globals["_PYTHONUDTF"]._serialized_start = 16459 + _globals["_PYTHONUDTF"]._serialized_end = 16636 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16639 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16790 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16792 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16867 + _globals["_COLLECTMETRICS"]._serialized_start = 16870 + _globals["_COLLECTMETRICS"]._serialized_end = 17006 + _globals["_PARSE"]._serialized_start = 17009 + _globals["_PARSE"]._serialized_end = 17397 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5861 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5919 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17298 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17386 + _globals["_ASOFJOIN"]._serialized_start = 17400 + _globals["_ASOFJOIN"]._serialized_end = 17875 + _globals["_LATERALJOIN"]._serialized_start = 17878 + _globals["_LATERALJOIN"]._serialized_end = 18108 # @@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 d4f1233e45f9b..cbf339ada2959 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -108,6 +108,7 @@ class Relation(google.protobuf.message.Message): TRANSPOSE_FIELD_NUMBER: builtins.int UNRESOLVED_TABLE_VALUED_FUNCTION_FIELD_NUMBER: builtins.int LATERAL_JOIN_FIELD_NUMBER: builtins.int + TRANSFORM_WITH_STATE_IN_PANDAS_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -216,6 +217,8 @@ class Relation(google.protobuf.message.Message): @property def lateral_join(self) -> global___LateralJoin: ... @property + def transform_with_state_in_pandas(self) -> global___TransformWithStateInPandas: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -301,6 +304,7 @@ class Relation(google.protobuf.message.Message): transpose: global___Transpose | None = ..., unresolved_table_valued_function: global___UnresolvedTableValuedFunction | None = ..., lateral_join: global___LateralJoin | None = ..., + transform_with_state_in_pandas: global___TransformWithStateInPandas | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -424,6 +428,8 @@ class Relation(google.protobuf.message.Message): b"to_df", "to_schema", b"to_schema", + "transform_with_state_in_pandas", + b"transform_with_state_in_pandas", "transpose", b"transpose", "unknown", @@ -549,6 +555,8 @@ class Relation(google.protobuf.message.Message): b"to_df", "to_schema", b"to_schema", + "transform_with_state_in_pandas", + b"transform_with_state_in_pandas", "transpose", b"transpose", "unknown", @@ -614,6 +622,7 @@ class Relation(google.protobuf.message.Message): "transpose", "unresolved_table_valued_function", "lateral_join", + "transform_with_state_in_pandas", "fill_na", "drop_na", "replace", @@ -3930,6 +3939,118 @@ class ApplyInPandasWithState(google.protobuf.message.Message): global___ApplyInPandasWithState = ApplyInPandasWithState +class TransformWithStateInPandas(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + INPUT_FIELD_NUMBER: builtins.int + GROUPING_EXPRESSIONS_FIELD_NUMBER: builtins.int + TRANSFORM_WITH_STATE_UDF_FIELD_NUMBER: builtins.int + OUTPUT_SCHEMA_FIELD_NUMBER: builtins.int + OUTPUT_MODE_FIELD_NUMBER: builtins.int + TIME_MODE_FIELD_NUMBER: builtins.int + INITIAL_INPUT_FIELD_NUMBER: builtins.int + INITIAL_GROUPING_EXPRESSIONS_FIELD_NUMBER: builtins.int + EVENT_TIME_COL_NAME_FIELD_NUMBER: builtins.int + @property + def input(self) -> global___Relation: + """(Required) Input relation for transformWithStateInPandas.""" + @property + def grouping_expressions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Required) Expressions for grouping keys.""" + @property + def transform_with_state_udf( + self, + ) -> pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction: + """(Required) Bytes for java serialized user-defined stateful processor.""" + output_schema: builtins.str + """(Required) Schema for the output DataFrame.""" + output_mode: builtins.str + """(Required) The output mode of the function.""" + time_mode: builtins.str + """(Required) Time mode for transformWithStateInPandas""" + @property + def initial_input(self) -> global___Relation: + """(Optional) Input relation for initial State.""" + @property + def initial_grouping_expressions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Optional) Expressions for grouping keys of the initial state input relation.""" + event_time_col_name: builtins.str + """(Optional) Event time column name""" + def __init__( + self, + *, + input: global___Relation | None = ..., + grouping_expressions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + transform_with_state_udf: pyspark.sql.connect.proto.expressions_pb2.CommonInlineUserDefinedFunction + | None = ..., + output_schema: builtins.str = ..., + output_mode: builtins.str = ..., + time_mode: builtins.str = ..., + initial_input: global___Relation | None = ..., + initial_grouping_expressions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + event_time_col_name: builtins.str | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_event_time_col_name", + b"_event_time_col_name", + "event_time_col_name", + b"event_time_col_name", + "initial_input", + b"initial_input", + "input", + b"input", + "transform_with_state_udf", + b"transform_with_state_udf", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_event_time_col_name", + b"_event_time_col_name", + "event_time_col_name", + b"event_time_col_name", + "grouping_expressions", + b"grouping_expressions", + "initial_grouping_expressions", + b"initial_grouping_expressions", + "initial_input", + b"initial_input", + "input", + b"input", + "output_mode", + b"output_mode", + "output_schema", + b"output_schema", + "time_mode", + b"time_mode", + "transform_with_state_udf", + b"transform_with_state_udf", + ], + ) -> None: ... + def WhichOneof( + self, + oneof_group: typing_extensions.Literal["_event_time_col_name", b"_event_time_col_name"], + ) -> typing_extensions.Literal["event_time_col_name"] | None: ... + +global___TransformWithStateInPandas = TransformWithStateInPandas + class CommonInlineUserDefinedTableFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py index cd78bf209b5e2..ea7041675b955 100644 --- a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -22,7 +22,7 @@ from pyspark.testing.connectutils import ReusedConnectTestCase -class TransformWithStateInPandasTests( +class TransformWithStateInPandasParityTests( TransformWithStateInPandasTestsMixin, ReusedConnectTestCase ): pass diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index d554a0cb37d73..da20b8e5124cf 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -59,7 +59,7 @@ class TransformWithStateInPandasTestsMixin: @classmethod def conf(cls): - cfg = SparkConf() + cfg = SparkConf(loadDefaults=False) # Avoid loading default configs cfg.set("spark.sql.shuffle.partitions", "5") cfg.set( "spark.sql.streaming.stateStore.providerClass", @@ -105,6 +105,9 @@ def _prepare_input_data_with_3_cols(self, input_path, col1, col2, col3): for e1, e2, e3 in zip(col1, col2, col3): fw.write(f"{e1},{e2},{e3}\n") + def end_query_from_feb_sink(self): + raise Exception(f"Ending the query by throw an exception for ProcessingTime mode") + def build_test_df_with_3_cols(self, input_path): df = self.spark.readStream.format("text").option("maxFilesPerTrigger", 1).load(input_path) df_split = df.withColumn("split_values", split(df["value"], ",")) @@ -182,6 +185,7 @@ def check_results(batch_df, batch_id): self._test_transform_with_state_in_pandas_basic(SimpleStatefulProcessor(), check_results) + """ def test_transform_with_state_in_pandas_non_exist_value_state(self): def check_results(batch_df, _): assert set(batch_df.sort("id").collect()) == { @@ -276,8 +280,7 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_basic( ListStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -301,8 +304,7 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_basic( MapStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -323,8 +325,7 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_basic( SimpleTTLStatefulProcessor(), check_results, False, "processingTime" @@ -384,8 +385,7 @@ def check_results(batch_df, batch_id): ], ) else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() if batch_id == 0 or batch_id == 1: time.sleep(4) @@ -519,8 +519,7 @@ def check_results(batch_df, batch_id): assert current_batch_expired_timestamp > self.first_expired_timestamp else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_proc_timer( ProcTimeStatefulProcessor(), check_results @@ -612,8 +611,7 @@ def check_results(batch_df, batch_id): Row(id="a-expired", timestamp="10000"), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_event_time( EventTimeStatefulProcessor(), check_results @@ -747,8 +745,7 @@ def check_results(batch_df, batch_id): Row(id1="1", id2="2", value=str(146 + 346)), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_non_contiguous_grouping_cols( SimpleStatefulProcessorWithInitialState(), check_results @@ -763,8 +760,7 @@ def check_results(batch_df, batch_id): Row(id1="1", id2="2", value=str(146 + 346)), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() # grouping key of initial state is also not starting from the beginning of attributes data = [(789, "0", "1"), (987, "3", "2")] @@ -883,8 +879,7 @@ def check_results(batch_df, batch_id): Row(id="3", value=str(987 + 12)), } else: - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_init_state_in_pandas( StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" @@ -1046,8 +1041,7 @@ def check_results(batch_df, batch_id): ) assert list_state_df.isEmpty() - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_basic( MapStateLargeTTLProcessor(), @@ -1140,8 +1134,7 @@ def check_results(batch_df, batch_id): Row(groupingKey="1", valueSortedList=[20, 20, 120, 120, 222]), ] - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() self._test_transform_with_state_in_pandas_basic( ListStateProcessor(), @@ -1230,8 +1223,7 @@ def check_results(batch_df, batch_id): .collect() ) - for q in self.spark.streams.active: - q.stop() + self.end_query_from_feb_sink() with self.sql_conf( {"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled": "true"} @@ -1354,7 +1346,7 @@ def _run_evolution_test( self.assertTrue(check_exception(e)) def test_schema_evolution_scenarios(self): - """Test various schema evolution scenarios""" + Test various schema evolution scenarios with self.sql_conf({"spark.sql.streaming.stateStore.encodingFormat": "avro"}): with tempfile.TemporaryDirectory() as checkpoint_dir: # Test 1: Basic state @@ -1469,7 +1461,7 @@ def check_exception(error): df, check_exception=check_exception, ) - + """ class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index 886d66e1e6e15..e17a17cdd05c5 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -1087,8 +1087,8 @@ message TransformWithStateInPandas { // (Required) Expressions for grouping keys. repeated Expression grouping_expressions = 2; - // (Required) Input user-defined function. - CommonInlineUserDefinedFunction func = 3; + // (Required) Bytes for java serialized user-defined stateful processor. + CommonInlineUserDefinedFunction transform_with_state_udf = 3; // (Required) Schema for the output DataFrame. string output_schema = 4; diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 8e683d89d5b4a..98f5211175d75 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -210,6 +210,8 @@ class SparkConnectPlanner( transformCoGroupMap(rel.getCoGroupMap) case proto.Relation.RelTypeCase.APPLY_IN_PANDAS_WITH_STATE => transformApplyInPandasWithState(rel.getApplyInPandasWithState) + case proto.Relation.RelTypeCase.TRANSFORM_WITH_STATE_IN_PANDAS => + transformTransformWithStateInPandas(rel.getTransformWithStateInPandas) case proto.Relation.RelTypeCase.COMMON_INLINE_USER_DEFINED_TABLE_FUNCTION => transformCommonInlineUserDefinedTableFunction( rel.getCommonInlineUserDefinedTableFunction) @@ -1021,6 +1023,25 @@ class SparkConnectPlanner( .logicalPlan } + private def transformTransformWithStateInPandas( + rel: proto.TransformWithStateInPandas): LogicalPlan = { + val pythonUdf = transformPythonUDF(rel.getTransformWithStateUdf) + val cols = + rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) + + val outputSchema = parseSchema(rel.getOutputSchema) + Dataset + .ofRows(session, transformRelation(rel.getInput)) + .groupBy(cols: _*) + .transformWithStateInPandas( + Column(pythonUdf), + outputSchema, + rel.getOutputMode, + rel.getTimeMode, + null, "") + .logicalPlan + } + private def transformCommonInlineUserDefinedTableFunction( fun: proto.CommonInlineUserDefinedTableFunction): LogicalPlan = { fun.getFunctionCase match { From b3569c109d4ccc8aac4eca0743435bc63254e275 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 23 Jan 2025 17:44:09 -0800 Subject: [PATCH 03/20] avoid accessing global spark session in feb sink --- python/pyspark/sql/connect/group.py | 2 +- python/pyspark/sql/pandas/group_ops.py | 2 +- .../test_pandas_transform_with_state.py | 43 ++++++++++++------- 3 files changed, 30 insertions(+), 17 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index d60a0e35c7af3..815c8be796988 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -499,7 +499,7 @@ def transformWithStateUDF( evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, ) - # TODO figure out if we need to handle for string type + # TODO add a string struct type test output_schema: str = ( outputStructType.json() if isinstance(outputStructType, StructType) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index e0108da34f0c2..b92b99569f306 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -519,7 +519,7 @@ def handle_pre_init( # won't be used again on JVM. statefulProcessor.close() - # return a dummy results, no return value is needed for pre init + # return a dummy result, no return value is needed for pre init return iter([]) def handle_data_rows( diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index da20b8e5124cf..fb43b36ba17d6 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -105,8 +105,11 @@ def _prepare_input_data_with_3_cols(self, input_path, col1, col2, col3): for e1, e2, e3 in zip(col1, col2, col3): fw.write(f"{e1},{e2},{e3}\n") - def end_query_from_feb_sink(self): - raise Exception(f"Ending the query by throw an exception for ProcessingTime mode") + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + def _check_query_end_exception(self, error): + error_msg = str(error) + return "Checks passed, ending the query for processing time mode" in error_msg def build_test_df_with_3_cols(self, input_path): df = self.spark.readStream.format("text").option("maxFilesPerTrigger", 1).load(input_path) @@ -285,7 +288,8 @@ def check_results(batch_df, batch_id): self._test_transform_with_state_in_pandas_basic( ListStateLargeTTLProcessor(), check_results, True, "processingTime" ) - + + """ def test_transform_with_state_in_pandas_map_state(self): def check_results(batch_df, _): assert set(batch_df.sort("id").collect()) == { @@ -294,6 +298,7 @@ def check_results(batch_df, _): } self._test_transform_with_state_in_pandas_basic(MapStateProcessor(), check_results, True) + """ # test map state with ttl has the same behavior as map state when state doesn't expire. def test_transform_with_state_in_pandas_map_state_large_ttl(self): @@ -1066,6 +1071,7 @@ def check_results(batch_df, batch_id): checkpoint_path=checkpoint_path, initial_state=initial_state, ) + """ # This test covers multiple list state variables and flatten option def test_transform_with_list_state_metadata(self): @@ -1079,7 +1085,7 @@ def check_results(batch_df, batch_id): } else: # check for state metadata source - metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) operator_properties_json_obj = json.loads( metadata_df.select("operatorProperties").collect()[0][0] ) @@ -1094,7 +1100,7 @@ def check_results(batch_df, batch_id): # check for state data source and flatten option list_state_1_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "listState1") .option("flattenCollectionTypes", True) @@ -1117,7 +1123,7 @@ def check_results(batch_df, batch_id): ] list_state_2_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "listState2") .option("flattenCollectionTypes", False) @@ -1134,16 +1140,23 @@ def check_results(batch_df, batch_id): Row(groupingKey="1", valueSortedList=[20, 20, 120, 120, 222]), ] - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_in_pandas_basic( - ListStateProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=None, - ) + try: + self._test_transform_with_state_in_pandas_basic( + ListStateProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) + + """ # This test covers value state variable and read change feed, # snapshotStartBatchId related options From 6e8cb25b00c43d40eb7c8cbf54ba2f90771fea58 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 24 Jan 2025 14:48:20 -0800 Subject: [PATCH 04/20] move udf into a shared file --- python/pyspark/sql/connect/group.py | 136 ++---------- python/pyspark/sql/pandas/group_ops.py | 194 +---------------- python/pyspark/sql/pandas/serializers.py | 3 +- .../sql/streaming/stateful_processor_util.py | 204 ++++++++++++++++++ 4 files changed, 223 insertions(+), 314 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 815c8be796988..6a1980d793f97 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -41,19 +41,7 @@ from pyspark.sql.column import Column from pyspark.sql.connect.functions import builtin as F from pyspark.errors import PySparkNotImplementedError, PySparkTypeError -from pyspark.sql.streaming.stateful_processor_api_client import ( - StatefulProcessorApiClient, - StatefulProcessorHandleState, -) -from pyspark.sql.streaming.stateful_processor import ( - ExpiredTimerInfo, - StatefulProcessor, - StatefulProcessorHandle, - TimerValues, -) from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle -from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode - if TYPE_CHECKING: from pyspark.sql.connect._typing import ( @@ -385,120 +373,22 @@ def transformWithStateInPandas( ) -> "DataFrame": from pyspark.sql.connect.udf import UserDefinedFunction from pyspark.sql.connect.dataframe import DataFrame - import itertools - from typing import Any, Iterator - - def handle_pre_init( - statefulProcessorApiClient: StatefulProcessorApiClient, - ) -> Iterator["PandasDataFrameLike"]: - # Driver handle is different from the handle used on executors; - # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which - # will only be used for handling init() and get the state schema on the driver. - driver_handle = StatefulProcessorHandle(statefulProcessorApiClient) - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.PRE_INIT) - statefulProcessor.init(driver_handle) - - # This method is used for the driver-side stateful processor after we have collected - # all the necessary schemas. This instance of the DriverStatefulProcessorHandleImpl - # won't be used again on JVM. - statefulProcessor.close() - - # return a dummy results, no return value is needed for pre init - return iter([]) - - def handle_data_rows( - statefulProcessorApiClient: StatefulProcessorApiClient, - key: Any, - inputRows: Optional[Iterator["PandasDataFrameLike"]] = None, - ) -> Iterator["PandasDataFrameLike"]: - statefulProcessorApiClient.set_implicit_key(key) - - batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( - timeMode + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasUdfUtils + + udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) + if initialState is None: + udf_obj = UserDefinedFunction( + udf_util.transformWithStateUDF, # type: ignore + returnType=outputStructType, + evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, ) - - # process with data rows - if inputRows is not None: - data_iter = statefulProcessor.handleInputRows( - key, inputRows, TimerValues(batch_timestamp, watermark_timestamp) - ) - return data_iter - else: - return iter([]) - - def handle_expired_timers( - statefulProcessorApiClient: StatefulProcessorApiClient, - ) -> Iterator["PandasDataFrameLike"]: - batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( - timeMode + else: + udf_obj = UserDefinedFunction( + udf_util.transformWithStateWithInitStateUDF, # type: ignore + returnType=outputStructType, + evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF, ) - if timeMode.lower() == "processingtime": - expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( - batch_timestamp - ) - elif timeMode.lower() == "eventtime": - expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( - watermark_timestamp - ) - else: - expiry_list_iter = iter([[]]) - - # process with expiry timers, only timer related rows will be emitted - for expiry_list in expiry_list_iter: - for key_obj, expiry_timestamp in expiry_list: - statefulProcessorApiClient.set_implicit_key(key_obj) - for pd in statefulProcessor.handleExpiredTimer( - key=key_obj, - timer_values=TimerValues(batch_timestamp, watermark_timestamp), - expired_timer_info=ExpiredTimerInfo(expiry_timestamp), - ): - yield pd - statefulProcessorApiClient.delete_timer(expiry_timestamp) - - def transformWithStateUDF( - statefulProcessorApiClient: StatefulProcessorApiClient, - mode: TransformWithStateInPandasFuncMode, - key: Any, - inputRows: Iterator["PandasDataFrameLike"], - ) -> Iterator["PandasDataFrameLike"]: - if mode == TransformWithStateInPandasFuncMode.PRE_INIT: - return handle_pre_init(statefulProcessorApiClient) - - handle = StatefulProcessorHandle(statefulProcessorApiClient) - - if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: - statefulProcessor.init(handle) - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.INITIALIZED - ) - - if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.DATA_PROCESSED - ) - result = handle_expired_timers(statefulProcessorApiClient) - return result - elif mode == TransformWithStateInPandasFuncMode.COMPLETE: - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.TIMER_PROCESSED - ) - statefulProcessorApiClient.remove_implicit_key() - statefulProcessor.close() - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) - return iter([]) - else: - # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA - result = handle_data_rows(statefulProcessorApiClient, key, inputRows) - return result - - # TODO add initial state - udf_obj = UserDefinedFunction( - transformWithStateUDF, - returnType=outputStructType, - evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, - ) - # TODO add a string struct type test output_schema: str = ( outputStructType.json() diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index b92b99569f306..e158258b295b3 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -24,18 +24,7 @@ from pyspark.sql.column import Column from pyspark.sql.dataframe import DataFrame from pyspark.sql.streaming.state import GroupStateTimeout -from pyspark.sql.streaming.stateful_processor_api_client import ( - StatefulProcessorApiClient, - StatefulProcessorHandleState, -) -from pyspark.sql.streaming.stateful_processor import ( - ExpiredTimerInfo, - StatefulProcessor, - StatefulProcessorHandle, - TimerValues, -) from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle -from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode from pyspark.sql.types import StructType if TYPE_CHECKING: @@ -46,7 +35,6 @@ PandasCogroupedMapFunction, ArrowGroupedMapFunction, ArrowCogroupedMapFunction, - DataFrameLike as PandasDataFrameLike, ) from pyspark.sql.group import GroupedData @@ -497,6 +485,7 @@ def transformWithStateInPandas( from pyspark.sql import GroupedData from pyspark.sql.functions import pandas_udf + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasUdfUtils assert isinstance(self, GroupedData) if initialState is not None: @@ -504,198 +493,23 @@ def transformWithStateInPandas( if isinstance(outputStructType, str): outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) - def handle_pre_init( - statefulProcessorApiClient: StatefulProcessorApiClient, - ) -> Iterator["PandasDataFrameLike"]: - # Driver handle is different from the handle used on executors; - # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which - # will only be used for handling init() and get the state schema on the driver. - driver_handle = StatefulProcessorHandle(statefulProcessorApiClient) - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.PRE_INIT) - statefulProcessor.init(driver_handle) - - # This method is used for the driver-side stateful processor after we have collected - # all the necessary schemas. This instance of the DriverStatefulProcessorHandleImpl - # won't be used again on JVM. - statefulProcessor.close() - - # return a dummy result, no return value is needed for pre init - return iter([]) - - def handle_data_rows( - statefulProcessorApiClient: StatefulProcessorApiClient, - key: Any, - inputRows: Optional[Iterator["PandasDataFrameLike"]] = None, - ) -> Iterator["PandasDataFrameLike"]: - statefulProcessorApiClient.set_implicit_key(key) - - batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( - timeMode - ) - - # process with data rows - if inputRows is not None: - data_iter = statefulProcessor.handleInputRows( - key, inputRows, TimerValues(batch_timestamp, watermark_timestamp) - ) - return data_iter - else: - return iter([]) - - def handle_expired_timers( - statefulProcessorApiClient: StatefulProcessorApiClient, - ) -> Iterator["PandasDataFrameLike"]: - batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( - timeMode - ) - - if timeMode.lower() == "processingtime": - expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( - batch_timestamp - ) - elif timeMode.lower() == "eventtime": - expiry_list_iter = statefulProcessorApiClient.get_expiry_timers_iterator( - watermark_timestamp - ) - else: - expiry_list_iter = iter([[]]) - - # process with expiry timers, only timer related rows will be emitted - for expiry_list in expiry_list_iter: - for key_obj, expiry_timestamp in expiry_list: - statefulProcessorApiClient.set_implicit_key(key_obj) - for pd in statefulProcessor.handleExpiredTimer( - key=key_obj, - timerValues=TimerValues(batch_timestamp, watermark_timestamp), - expiredTimerInfo=ExpiredTimerInfo(expiry_timestamp), - ): - yield pd - statefulProcessorApiClient.delete_timer(expiry_timestamp) - - def transformWithStateUDF( - statefulProcessorApiClient: StatefulProcessorApiClient, - mode: TransformWithStateInPandasFuncMode, - key: Any, - inputRows: Iterator["PandasDataFrameLike"], - ) -> Iterator["PandasDataFrameLike"]: - if mode == TransformWithStateInPandasFuncMode.PRE_INIT: - return handle_pre_init(statefulProcessorApiClient) - - handle = StatefulProcessorHandle(statefulProcessorApiClient) - - if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: - statefulProcessor.init(handle) - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.INITIALIZED - ) - - if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.DATA_PROCESSED - ) - result = handle_expired_timers(statefulProcessorApiClient) - return result - elif mode == TransformWithStateInPandasFuncMode.COMPLETE: - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.TIMER_PROCESSED - ) - statefulProcessorApiClient.remove_implicit_key() - statefulProcessor.close() - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) - return iter([]) - else: - # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA - result = handle_data_rows(statefulProcessorApiClient, key, inputRows) - return result - - def transformWithStateWithInitStateUDF( - statefulProcessorApiClient: StatefulProcessorApiClient, - mode: TransformWithStateInPandasFuncMode, - key: Any, - inputRows: Iterator["PandasDataFrameLike"], - initialStates: Optional[Iterator["PandasDataFrameLike"]] = None, - ) -> Iterator["PandasDataFrameLike"]: - """ - UDF for TWS operator with non-empty initial states. Possible input combinations - of inputRows and initialStates iterator: - - Both `inputRows` and `initialStates` are non-empty. Both input rows and initial - states contains the grouping key and data. - - `InitialStates` is non-empty, while `inputRows` is empty. Only initial states - contains the grouping key and data, and it is first batch. - - `initialStates` is empty, while `inputRows` is non-empty. Only inputRows contains the - grouping key and data, and it is first batch. - - `initialStates` is None, while `inputRows` is not empty. This is not first batch. - `initialStates` is initialized to the positional value as None. - """ - if mode == TransformWithStateInPandasFuncMode.PRE_INIT: - return handle_pre_init(statefulProcessorApiClient) - - handle = StatefulProcessorHandle(statefulProcessorApiClient) - - if statefulProcessorApiClient.handle_state == StatefulProcessorHandleState.CREATED: - statefulProcessor.init(handle) - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.INITIALIZED - ) - - if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: - statefulProcessorApiClient.set_handle_state( - StatefulProcessorHandleState.DATA_PROCESSED - ) - result = handle_expired_timers(statefulProcessorApiClient) - return result - elif mode == TransformWithStateInPandasFuncMode.COMPLETE: - statefulProcessorApiClient.remove_implicit_key() - statefulProcessor.close() - statefulProcessorApiClient.set_handle_state(StatefulProcessorHandleState.CLOSED) - return iter([]) - else: - # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA - batch_timestamp, watermark_timestamp = statefulProcessorApiClient.get_timestamps( - timeMode - ) - - # only process initial state if first batch and initial state is not None - if initialStates is not None: - for cur_initial_state in initialStates: - statefulProcessorApiClient.set_implicit_key(key) - statefulProcessor.handleInitialState( - key, cur_initial_state, TimerValues(batch_timestamp, watermark_timestamp) - ) - - # if we don't have input rows for the given key but only have initial state - # for the grouping key, the inputRows iterator could be empty - input_rows_empty = False - try: - first = next(inputRows) - except StopIteration: - input_rows_empty = True - else: - inputRows = itertools.chain([first], inputRows) - - if not input_rows_empty: - result = handle_data_rows(statefulProcessorApiClient, key, inputRows) - else: - result = iter([]) - - return result - if isinstance(outputStructType, str): outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) df = self._df + udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) if initialState is None: initial_state_java_obj = None udf = pandas_udf( - transformWithStateUDF, # type: ignore + udf_util.transformWithStateUDF, # type: ignore returnType=outputStructType, functionType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, ) else: initial_state_java_obj = initialState._jgd udf = pandas_udf( - transformWithStateWithInitStateUDF, # type: ignore + udf_util.transformWithStateWithInitStateUDF, # type: ignore returnType=outputStructType, functionType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF, ) diff --git a/python/pyspark/sql/pandas/serializers.py b/python/pyspark/sql/pandas/serializers.py index 536bf7307065c..9320358a14af0 100644 --- a/python/pyspark/sql/pandas/serializers.py +++ b/python/pyspark/sql/pandas/serializers.py @@ -36,7 +36,6 @@ _create_converter_from_pandas, _create_converter_to_pandas, ) -from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode from pyspark.sql.types import ( DataType, StringType, @@ -1175,6 +1174,7 @@ def load_stream(self, stream): this function works in overall. """ import pyarrow as pa + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode def generate_data_batches(batches): """ @@ -1230,6 +1230,7 @@ def __init__(self, timezone, safecheck, assign_cols_by_name, arrow_max_records_p def load_stream(self, stream): import pyarrow as pa + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode def generate_data_batches(batches): """ diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index d69c1a943862c..a8e91337eba91 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -16,6 +16,22 @@ # from enum import Enum +import itertools +from typing import Any, Iterator, List, Optional, Union, TYPE_CHECKING, cast +from pyspark.sql.streaming.stateful_processor_api_client import ( + StatefulProcessorApiClient, + StatefulProcessorHandleState, +) +from pyspark.sql.streaming.stateful_processor import ( + ExpiredTimerInfo, + StatefulProcessor, + StatefulProcessorHandle, + TimerValues, +) +from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle +if TYPE_CHECKING: + from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode # This file places the utilities for transformWithStateInPandas; we have a separate file to avoid # putting internal classes to the stateful_processor.py file which contains public APIs. @@ -26,3 +42,191 @@ class TransformWithStateInPandasFuncMode(Enum): PROCESS_TIMER = 2 COMPLETE = 3 PRE_INIT = 4 + + +class TransformWithStateInPandasUdfUtils: + + def __init__(self, stateful_processor: StatefulProcessor, time_mode: str): + self._stateful_processor = stateful_processor + self._time_mode = time_mode + + def transformWithStateUDF( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + input_rows: Iterator["PandasDataFrameLike"], + ) -> Iterator["PandasDataFrameLike"]: + if mode == TransformWithStateInPandasFuncMode.PRE_INIT: + return self._handle_pre_init(stateful_processor_api_client) + + handle = StatefulProcessorHandle(stateful_processor_api_client) + + if stateful_processor_api_client.handle_state == StatefulProcessorHandleState.CREATED: + self._stateful_processor.init(handle) + stateful_processor_api_client.set_handle_state( + StatefulProcessorHandleState.INITIALIZED + ) + + if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: + stateful_processor_api_client.set_handle_state( + StatefulProcessorHandleState.DATA_PROCESSED + ) + result = self._handle_expired_timers(stateful_processor_api_client) + return result + elif mode == TransformWithStateInPandasFuncMode.COMPLETE: + stateful_processor_api_client.set_handle_state( + StatefulProcessorHandleState.TIMER_PROCESSED + ) + stateful_processor_api_client.remove_implicit_key() + self._stateful_processor.close() + stateful_processor_api_client.set_handle_state(StatefulProcessorHandleState.CLOSED) + return iter([]) + else: + # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA + result = self._handle_data_rows(stateful_processor_api_client, key, input_rows) + return result + + def transformWithStateWithInitStateUDF( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + input_rows: Iterator["PandasDataFrameLike"], + initial_states: Optional[Iterator["PandasDataFrameLike"]] = None, + ) -> Iterator["PandasDataFrameLike"]: + """ + UDF for TWS operator with non-empty initial states. Possible input combinations + of inputRows and initialStates iterator: + - Both `inputRows` and `initialStates` are non-empty. Both input rows and initial + states contains the grouping key and data. + - `InitialStates` is non-empty, while `inputRows` is empty. Only initial states + contains the grouping key and data, and it is first batch. + - `initialStates` is empty, while `inputRows` is non-empty. Only inputRows contains the + grouping key and data, and it is first batch. + - `initialStates` is None, while `inputRows` is not empty. This is not first batch. + `initialStates` is initialized to the positional value as None. + """ + if mode == TransformWithStateInPandasFuncMode.PRE_INIT: + return self._handle_pre_init(stateful_processor_api_client) + + handle = StatefulProcessorHandle(stateful_processor_api_client) + + if stateful_processor_api_client.handle_state == StatefulProcessorHandleState.CREATED: + self._stateful_processor.init(handle) + stateful_processor_api_client.set_handle_state( + StatefulProcessorHandleState.INITIALIZED + ) + + if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: + stateful_processor_api_client.set_handle_state( + StatefulProcessorHandleState.DATA_PROCESSED + ) + result = self._handle_expired_timers(stateful_processor_api_client) + return result + elif mode == TransformWithStateInPandasFuncMode.COMPLETE: + stateful_processor_api_client.remove_implicit_key() + self._stateful_processor.close() + stateful_processor_api_client.set_handle_state(StatefulProcessorHandleState.CLOSED) + return iter([]) + else: + # mode == TransformWithStateInPandasFuncMode.PROCESS_DATA + batch_timestamp, watermark_timestamp = stateful_processor_api_client.get_timestamps( + self._time_mode + ) + + # only process initial state if first batch and initial state is not None + if initial_states is not None: + for cur_initial_state in initial_states: + stateful_processor_api_client.set_implicit_key(key) + self._stateful_processor.handleInitialState( + key, cur_initial_state, TimerValues(batch_timestamp, watermark_timestamp) + ) + + # if we don't have input rows for the given key but only have initial state + # for the grouping key, the inputRows iterator could be empty + input_rows_empty = False + try: + first = next(input_rows) + except StopIteration: + input_rows_empty = True + else: + input_rows = itertools.chain([first], input_rows) + + if not input_rows_empty: + result = self._handle_data_rows(stateful_processor_api_client, key, input_rows) + else: + result = iter([]) + + return result + + def _handle_pre_init( + self, + stateful_processor_api_client: StatefulProcessorApiClient + ) -> Iterator["PandasDataFrameLike"]: + # Driver handle is different from the handle used on executors; + # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which + # will only be used for handling init() and get the state schema on the driver. + driver_handle = StatefulProcessorHandle(stateful_processor_api_client) + stateful_processor_api_client.set_handle_state(StatefulProcessorHandleState.PRE_INIT) + self._stateful_processor.init(driver_handle) + + # This method is used for the driver-side stateful processor after we have collected + # all the necessary schemas. This instance of the DriverStatefulProcessorHandleImpl + # won't be used again on JVM. + self._stateful_processor.close() + + # return a dummy result, no return value is needed for pre init + return iter([]) + + def _handle_data_rows( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + key: Any, + input_rows: Optional[Iterator["PandasDataFrameLike"]] = None, + ) -> Iterator["PandasDataFrameLike"]: + stateful_processor_api_client.set_implicit_key(key) + + batch_timestamp, watermark_timestamp = stateful_processor_api_client.get_timestamps( + self._time_mode + ) + + # process with data rows + if input_rows is not None: + data_iter = self._stateful_processor.handleInputRows( + key, input_rows, TimerValues(batch_timestamp, watermark_timestamp) + ) + return data_iter + else: + return iter([]) + + def _handle_expired_timers( + self, + stateful_processor_api_client: StatefulProcessorApiClient, + ) -> Iterator["PandasDataFrameLike"]: + batch_timestamp, watermark_timestamp = stateful_processor_api_client.get_timestamps( + self._time_mode + ) + + if self._time_mode.lower() == "processingtime": + expiry_list_iter = stateful_processor_api_client.get_expiry_timers_iterator( + batch_timestamp + ) + elif self._time_mode.lower() == "eventtime": + expiry_list_iter = stateful_processor_api_client.get_expiry_timers_iterator( + watermark_timestamp + ) + else: + expiry_list_iter = iter([[]]) + + # process with expiry timers, only timer related rows will be emitted + for expiry_list in expiry_list_iter: + for key_obj, expiry_timestamp in expiry_list: + stateful_processor_api_client.set_implicit_key(key_obj) + for pd in self._stateful_processor.handleExpiredTimer( + key=key_obj, + timerValues=TimerValues(batch_timestamp, watermark_timestamp), + expiredTimerInfo=ExpiredTimerInfo(expiry_timestamp), + ): + yield pd + stateful_processor_api_client.delete_timer(expiry_timestamp) \ No newline at end of file From 13acf55a75ed1de5c750c3c3b665b8a7151c1b31 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 27 Jan 2025 16:41:11 -0800 Subject: [PATCH 05/20] initial state work --- python/pyspark/sql/connect/group.py | 9 + python/pyspark/sql/connect/plan.py | 23 +- python/pyspark/sql/pandas/group_ops.py | 1 - .../tests/pandas/test_pandas_cogrouped_map.py | 4 + .../test_pandas_transform_with_state.py | 209 +++++++++++------- .../logical/pythonLogicalOperators.scala | 1 + .../connect/planner/SparkConnectPlanner.scala | 48 +++- .../classic/RelationalGroupedDataset.scala | 23 +- 8 files changed, 221 insertions(+), 97 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 6a1980d793f97..1f86b869f8fae 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -375,6 +375,8 @@ def transformWithStateInPandas( from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasUdfUtils + self._df._check_same_session(initialState._df) + udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) if initialState is None: udf_obj = UserDefinedFunction( @@ -382,12 +384,17 @@ def transformWithStateInPandas( returnType=outputStructType, evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, ) + initial_state_plan = None + initial_state_grouping_cols = None + else: udf_obj = UserDefinedFunction( udf_util.transformWithStateWithInitStateUDF, # type: ignore returnType=outputStructType, evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF, ) + initial_state_plan = initialState._df._plan + initial_state_grouping_cols = initialState._grouping_cols # TODO add a string struct type test output_schema: str = ( @@ -405,6 +412,8 @@ def transformWithStateInPandas( output_mode=outputMode, time_mode=timeMode, cols=self._df.columns, + initial_state_plan=initial_state_plan, + initial_state_grouping_cols=initial_state_grouping_cols, ), session=self._df._session, ) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 26b6f85bd4795..84abe3578bd2e 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2559,15 +2559,22 @@ def __init__( output_mode: str, time_mode: str, cols: List[str], + initial_state_plan: Optional["LogicalPlan"], + initial_state_grouping_cols: Optional[Sequence[Column]], ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) + assert isinstance(initial_state_grouping_cols, list)\ + and all(isinstance(c, Column) for c in initial_state_grouping_cols) - super().__init__(child, self._collect_references(grouping_cols)) + super().__init__(child, self._collect_references(grouping_cols + initial_state_grouping_cols)) + # raise Exception(f"collect references: {self._collect_references(grouping_cols + initial_state_grouping_cols)}") self._grouping_cols = grouping_cols self._output_schema = output_schema self._output_mode = output_mode self._time_mode = time_mode self._function = function._build_common_inline_user_defined_function(*cols) + self._initial_state_plan = initial_state_plan + self._initial_state_grouping_cols = initial_state_grouping_cols def plan(self, session: "SparkConnectClient") -> proto.Relation: assert self._child is not None @@ -2576,13 +2583,21 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.transform_with_state_in_pandas.grouping_expressions.extend( [c.to_plan(session) for c in self._grouping_cols] ) + # fill in initial state related fields + if self._initial_state_plan is not None: + self._initial_state_plan = cast(LogicalPlan, self._initial_state_plan) + plan.transform_with_state_in_pandas.initial_input.CopyFrom(self._initial_state_plan.plan(session)) + plan.transform_with_state_in_pandas.initial_grouping_expressions.extend( + [c.to_plan(session) for c in self._initial_state_grouping_cols] + ) - # this is to deserialize stateful processor - plan.transform_with_state_in_pandas.transform_with_state_udf.CopyFrom( - self._function.to_plan_udf(session)) + # wrap transformWithStateInPandasUdf in a function plan.transform_with_state_in_pandas.output_schema = self._output_schema plan.transform_with_state_in_pandas.output_mode = self._output_mode plan.transform_with_state_in_pandas.time_mode = self._time_mode + plan.transform_with_state_in_pandas.transform_with_state_udf.CopyFrom( + self._function.to_plan_udf(session)) + return self._with_relations(plan, session) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index e158258b295b3..45171966afaf1 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -14,7 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -import itertools import sys from typing import Any, Iterator, List, Optional, Union, TYPE_CHECKING, cast import warnings diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index 1f9532352679a..55f3ed267e583 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -71,6 +71,7 @@ def data2(self): .drop("ks") ) + """ def test_simple(self): self._test_merge(self.data1, self.data2) @@ -356,6 +357,7 @@ def check_wrong_args(self): errorClass=ValueError, error_message_regex="Invalid function", ) + """ def test_case_insensitive_grouping_column(self): # SPARK-31915: case-insensitive grouping column should work. @@ -379,6 +381,7 @@ def test_case_insensitive_grouping_column(self): ) self.assertEqual(row.asDict(), Row(column=2, value=2).asDict()) + """ def test_self_join(self): # SPARK-34319: self-join with FlatMapCoGroupsInPandas df = self.spark.createDataFrame([(1, 1)], ("column", "value")) @@ -478,6 +481,7 @@ def summarize(left, right): "| 2| 3| 1| 2|\n" "+---------+------------+----------+-------------+\n", ) + """ @staticmethod def _test_with_key(left, right, isLeft): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index fb43b36ba17d6..88a897aa32d5b 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -121,6 +121,7 @@ def build_test_df_with_3_cols(self, input_path): ) return df_final + """ def _test_transform_with_state_in_pandas_basic( self, stateful_processor, @@ -188,7 +189,6 @@ def check_results(batch_df, batch_id): self._test_transform_with_state_in_pandas_basic(SimpleStatefulProcessor(), check_results) - """ def test_transform_with_state_in_pandas_non_exist_value_state(self): def check_results(batch_df, _): assert set(batch_df.sort("id").collect()) == { @@ -283,13 +283,17 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") + + try: + self._test_transform_with_state_in_pandas_basic( + ListStateLargeTTLProcessor(), check_results, True, "processingTime" + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) - self._test_transform_with_state_in_pandas_basic( - ListStateLargeTTLProcessor(), check_results, True, "processingTime" - ) - - """ def test_transform_with_state_in_pandas_map_state(self): def check_results(batch_df, _): assert set(batch_df.sort("id").collect()) == { @@ -298,7 +302,6 @@ def check_results(batch_df, _): } self._test_transform_with_state_in_pandas_basic(MapStateProcessor(), check_results, True) - """ # test map state with ttl has the same behavior as map state when state doesn't expire. def test_transform_with_state_in_pandas_map_state_large_ttl(self): @@ -309,11 +312,16 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), check_results, True, "processingTime" - ) + try: + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), check_results, True, "processingTime" + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) # test value state with ttl has the same behavior as value state when # state doesn't expire. @@ -330,11 +338,15 @@ def check_results(batch_df, batch_id): Row(id="1", countAsString="2"), } else: - self.end_query_from_feb_sink() - - self._test_transform_with_state_in_pandas_basic( - SimpleTTLStatefulProcessor(), check_results, False, "processingTime" - ) + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") + try: + self._test_transform_with_state_in_pandas_basic( + SimpleTTLStatefulProcessor(), check_results, False, "processingTime" + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) # TODO SPARK-50908 holistic fix for TTL suite @unittest.skip("test is flaky and it is only a timing issue, skipping until we can resolve") @@ -390,7 +402,9 @@ def check_results(batch_df, batch_id): ], ) else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") if batch_id == 0 or batch_id == 1: time.sleep(4) @@ -427,6 +441,8 @@ def check_results(batch_df, batch_id): q.stop() q.awaitTermination() self.assertTrue(q.exception() is None) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) finally: input_dir.cleanup() @@ -473,6 +489,7 @@ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, ch q.awaitTermination(10) self.assertTrue(q.exception() is None) + # TODO fix later def test_transform_with_state_in_pandas_proc_timer(self): # helper function to check expired timestamp is smaller than current processing time def check_timestamp(batch_df): @@ -507,7 +524,7 @@ def check_results(batch_df, batch_id): self.first_expired_timestamp = batch_df.filter( batch_df["countAsString"] == -1 ).first()["timeValues"] - check_timestamp(batch_df) + # check_timestamp(batch_df) elif batch_id == 2: assert set(batch_df.sort("id").select("id", "countAsString").collect()) == { @@ -521,14 +538,19 @@ def check_results(batch_df, batch_id): current_batch_expired_timestamp = batch_df.filter( batch_df["countAsString"] == -1 ).first()["timeValues"] - assert current_batch_expired_timestamp > self.first_expired_timestamp + # assert current_batch_expired_timestamp > self.first_expired_timestamp else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_in_pandas_proc_timer( - ProcTimeStatefulProcessor(), check_results - ) + try: + self._test_transform_with_state_in_pandas_proc_timer( + ProcTimeStatefulProcessor(), check_results + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) def _test_transform_with_state_in_pandas_event_time(self, stateful_processor, check_results): import pyspark.sql.functions as f @@ -616,11 +638,17 @@ def check_results(batch_df, batch_id): Row(id="a-expired", timestamp="10000"), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_in_pandas_event_time( - EventTimeStatefulProcessor(), check_results - ) + try: + self._test_transform_with_state_in_pandas_event_time( + EventTimeStatefulProcessor(), check_results + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) + """ def _test_transform_with_state_init_state_in_pandas( self, @@ -699,6 +727,7 @@ def check_results(batch_df, batch_id): SimpleStatefulProcessorWithInitialState(), check_results ) + """ def _test_transform_with_state_non_contiguous_grouping_cols( self, stateful_processor, check_results, initial_state=None ): @@ -750,11 +779,16 @@ def check_results(batch_df, batch_id): Row(id1="1", id2="2", value=str(146 + 346)), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_non_contiguous_grouping_cols( - SimpleStatefulProcessorWithInitialState(), check_results - ) + try: + self._test_transform_with_state_non_contiguous_grouping_cols( + SimpleStatefulProcessorWithInitialState(), check_results + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) def test_transform_with_state_non_contiguous_grouping_cols_with_init_state(self): def check_results(batch_df, batch_id): @@ -765,7 +799,9 @@ def check_results(batch_df, batch_id): Row(id1="1", id2="2", value=str(146 + 346)), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") # grouping key of initial state is also not starting from the beginning of attributes data = [(789, "0", "1"), (987, "3", "2")] @@ -773,10 +809,14 @@ def check_results(batch_df, batch_id): data, "initVal int, id1 string, id2 string" ).groupBy("id1", "id2") - self._test_transform_with_state_non_contiguous_grouping_cols( - SimpleStatefulProcessorWithInitialState(), check_results, initial_state - ) + try: + self._test_transform_with_state_non_contiguous_grouping_cols( + SimpleStatefulProcessorWithInitialState(), check_results, initial_state + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) + @unittest.skip("chaining of ops not supported yet") def _test_transform_with_state_in_pandas_chaining_ops( self, stateful_processor, check_results, timeMode="None", grouping_cols=["outputTimestamp"] ): @@ -884,11 +924,16 @@ def check_results(batch_df, batch_id): Row(id="3", value=str(987 + 12)), } else: - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_init_state_in_pandas( - StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" - ) + try: + self._test_transform_with_state_init_state_in_pandas( + StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) def test_transform_with_state_in_pandas_batch_query(self): data = [("0", 123), ("0", 46), ("1", 146), ("1", 346)] @@ -954,7 +999,7 @@ def check_results(batch_df, batch_id): } else: # check for state metadata source - metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) assert set( metadata_df.select( "operatorId", @@ -995,7 +1040,7 @@ def check_results(batch_df, batch_id): # check for state data source map_state_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "mapState") .load() @@ -1011,7 +1056,7 @@ def check_results(batch_df, batch_id): # check for map state with flatten option map_state_df_non_flatten = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "mapState") .option("flattenCollectionTypes", False) @@ -1039,23 +1084,28 @@ def check_results(batch_df, batch_id): assert len(set(ttl_df)) == 1 list_state_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "listState") .load() ) assert list_state_df.isEmpty() - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=None, - ) + try: + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) # run the same test suite again but with no-op initial state # TWS with initial state is using a different python runner @@ -1063,15 +1113,17 @@ def check_results(batch_df, batch_id): initial_state = self.spark.createDataFrame(init_data, "id string, temperature int").groupBy( "id" ) - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=initial_state, - ) - """ + try: + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=initial_state, + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) # This test covers multiple list state variables and flatten option def test_transform_with_list_state_metadata(self): @@ -1156,8 +1208,6 @@ def check_results(batch_df, batch_id): except Exception as e: self.assertTrue(self._check_query_end_exception(e)) - """ - # This test covers value state variable and read change feed, # snapshotStartBatchId related options def test_transform_with_value_state_metadata(self): @@ -1176,7 +1226,7 @@ def check_results(batch_df, batch_id): } # check for state metadata source - metadata_df = self.spark.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) operator_properties_json_obj = json.loads( metadata_df.select("operatorProperties").collect()[0][0] ) @@ -1192,7 +1242,7 @@ def check_results(batch_df, batch_id): # check for state data source and readChangeFeed value_state_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "numViolations") .option("readChangeFeed", True) @@ -1216,7 +1266,7 @@ def check_results(batch_df, batch_id): for partition_id in partition_id_list: # check for state data source and snapshotStartBatchId options state_snapshot_df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", checkpoint_path) .option("stateVarName", "numViolations") .option("snapshotPartitionId", partition_id) @@ -1236,18 +1286,23 @@ def check_results(batch_df, batch_id): .collect() ) - self.end_query_from_feb_sink() + # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed + # for processing time mode + raise Exception("Checks passed, ending the query for processing time mode") with self.sql_conf( {"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled": "true"} ): - self._test_transform_with_state_in_pandas_basic( - SimpleStatefulProcessor(), - check_results, - False, - "processingTime", - checkpoint_path=checkpoint_path, - ) + try: + self._test_transform_with_state_in_pandas_basic( + SimpleStatefulProcessor(), + check_results, + False, + "processingTime", + checkpoint_path=checkpoint_path, + ) + except Exception as e: + self.assertTrue(self._check_query_end_exception(e)) def test_transform_with_state_restart_with_multiple_rows_init_state(self): def check_results(batch_df, _): @@ -1269,7 +1324,7 @@ def check_results_for_new_query(batch_df, batch_id): } # verify values in initial state is appended into list state for all keys df = ( - self.spark.read.format("statestore") + batch_df.sparkSession.read.format("statestore") .option("path", new_checkpoint_path) .option("stateVarName", "list_state") .load() @@ -1359,7 +1414,7 @@ def _run_evolution_test( self.assertTrue(check_exception(e)) def test_schema_evolution_scenarios(self): - Test various schema evolution scenarios + # Test various schema evolution scenarios with self.sql_conf({"spark.sql.streaming.stateStore.encodingFormat": "avro"}): with tempfile.TemporaryDirectory() as checkpoint_dir: # Test 1: Basic state diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 2f9bf2b52190a..3ab5accee2a25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -240,6 +240,7 @@ case class FlatMapCoGroupsInArrow( override val producedAttributes = AttributeSet(output) override lazy val references: AttributeSet = AttributeSet(leftAttributes ++ rightAttributes ++ functionExpr.references) -- producedAttributes + throw new Exception(s"what is the references here: ${references.toSeq}") def leftAttributes: Seq[Attribute] = left.output.take(leftGroupingLen) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 98f5211175d75..a5a57c8d82f1e 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1030,16 +1030,44 @@ class SparkConnectPlanner( rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) val outputSchema = parseSchema(rel.getOutputSchema) - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .groupBy(cols: _*) - .transformWithStateInPandas( - Column(pythonUdf), - outputSchema, - rel.getOutputMode, - rel.getTimeMode, - null, "") - .logicalPlan + + if (rel.hasInitialInput) { + val initialGroupingCols = rel.getInitialGroupingExpressionsList.asScala.toSeq.map( + expr => Column(transformExpression(expr))) + + val input = Dataset + .ofRows(session, transformRelation(rel.getInput)) + .groupBy(cols: _*) + val initialStateDs = Dataset + .ofRows(session, transformRelation(rel.getInitialInput)) + .groupBy(initialGroupingCols: _*) + + // Explicitly creating UDF on resolved column to avoid ambiguity of analysis on initial state + // columns and the input columns + val resolvedPythonUDF = createUserDefinedPythonFunction(rel.getTransformWithStateUdf) + .builder(input.df.logicalPlan.output) + .asInstanceOf[PythonUDF] + + input + .transformWithStateInPandas( + Column(resolvedPythonUDF), + outputSchema, + rel.getOutputMode, + rel.getTimeMode, + initialStateDs, "") + .logicalPlan + } else { + Dataset + .ofRows(session, transformRelation(rel.getInput)) + .groupBy(cols: _*) + .transformWithStateInPandas( + Column(pythonUdf), + outputSchema, + rel.getOutputMode, + rel.getTimeMode, + null, "") + .logicalPlan + } } private def transformCommonInlineUserDefinedTableFunction( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala index 082292145e858..c9925749786f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala @@ -348,12 +348,16 @@ class RelationalGroupedDataset protected[sql]( val leftGroupingNamedExpressions = groupingExprs.map { case ne: NamedExpression => ne - case other => Alias(other, other.toString)() + case other => + throw new Exception(s"I am here, attaching alias") + Alias(other, other.toString)() } val rightGroupingNamedExpressions = r.groupingExprs.map { case ne: NamedExpression => ne - case other => Alias(other, other.toString)() + case other => + throw new Exception(s"I am here, attaching alias") + Alias(other, other.toString)() } val leftChild = df.logicalPlan @@ -363,6 +367,13 @@ class RelationalGroupedDataset protected[sql]( Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild)).analyzed val right = r.df.sparkSession.sessionState.executePlan( Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild)).analyzed + /* + throw new Exception(f"initial state grouping expr: ${r.groupingExprs}, " + + f"initGroupingAttrs: $rightGroupingNamedExpressions, df grouping expr: ${groupingExprs}, " + + f"df grouping attrs: ${leftGroupingNamedExpressions}") + + throw new Exception(f"rightChild plan: ${rightChild}," + + f" rightGroupingNamedExpressions: $rightGroupingNamedExpressions, right: $right") */ val output = toAttributes(expr.dataType.asInstanceOf[StructType]) val plan = FlatMapCoGroupsInPandas( @@ -479,11 +490,13 @@ class RelationalGroupedDataset protected[sql]( timeModeStr: String, initialState: RelationalGroupedDataset, eventTimeColumnName: String): DataFrame = { - def exprToAttr(expr: Seq[Expression]): Seq[Attribute] = { + def exprToAttr(expr: Seq[Expression]) = { expr.map { case ne: NamedExpression => ne - case other => Alias(other, other.toString)() - }.map(_.toAttribute) + case other => + println(s"I am here, attaching alias") + Alias(other, other.toString)() + } } val groupingAttrs = exprToAttr(groupingExprs) From f700da85390a67c6be7d0dafe0d9dec4f17f22d9 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 27 Jan 2025 17:28:32 -0800 Subject: [PATCH 06/20] fix most test cases --- python/pyspark/sql/connect/group.py | 18 +- python/pyspark/sql/connect/plan.py | 38 +-- .../test_pandas_transform_with_state.py | 261 ++++++------------ .../connect/planner/SparkConnectPlanner.scala | 4 +- .../classic/RelationalGroupedDataset.scala | 23 +- 5 files changed, 119 insertions(+), 225 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 1f86b869f8fae..37b6bff09dc3b 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -363,20 +363,18 @@ def applyInPandasWithState( applyInPandasWithState.__doc__ = PySparkGroupedData.applyInPandasWithState.__doc__ def transformWithStateInPandas( - self, - statefulProcessor: StatefulProcessor, - outputStructType: Union[StructType, str], - outputMode: str, - timeMode: str, - initialState: Optional["GroupedData"] = None, - eventTimeColumnName: str = "", + self, + statefulProcessor: StatefulProcessor, + outputStructType: Union[StructType, str], + outputMode: str, + timeMode: str, + initialState: Optional["GroupedData"] = None, + eventTimeColumnName: str = "", ) -> "DataFrame": from pyspark.sql.connect.udf import UserDefinedFunction from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasUdfUtils - self._df._check_same_session(initialState._df) - udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) if initialState is None: udf_obj = UserDefinedFunction( @@ -388,6 +386,7 @@ def transformWithStateInPandas( initial_state_grouping_cols = None else: + self._df._check_same_session(initialState._df) udf_obj = UserDefinedFunction( udf_util.transformWithStateWithInitStateUDF, # type: ignore returnType=outputStructType, @@ -411,6 +410,7 @@ def transformWithStateInPandas( output_schema=output_schema, output_mode=outputMode, time_mode=timeMode, + event_time_col_name=eventTimeColumnName, cols=self._df.columns, initial_state_plan=initial_state_plan, initial_state_grouping_cols=initial_state_grouping_cols, diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 84abe3578bd2e..58553facfb09a 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2548,30 +2548,33 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: class TransformWithStateInPandas(LogicalPlan): - """Logical plan object for a applyInPandasWithState.""" + """Logical plan object for a TransformWithStateInPandas.""" def __init__( - self, - child: Optional["LogicalPlan"], - grouping_cols: Sequence[Column], - function: "UserDefinedFunction", - output_schema: str, - output_mode: str, - time_mode: str, - cols: List[str], - initial_state_plan: Optional["LogicalPlan"], - initial_state_grouping_cols: Optional[Sequence[Column]], + self, + child: Optional["LogicalPlan"], + grouping_cols: Sequence[Column], + function: "UserDefinedFunction", + output_schema: str, + output_mode: str, + time_mode: str, + event_time_col_name: str, + cols: List[str], + initial_state_plan: Optional["LogicalPlan"], + initial_state_grouping_cols: Optional[Sequence[Column]], ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) - assert isinstance(initial_state_grouping_cols, list)\ - and all(isinstance(c, Column) for c in initial_state_grouping_cols) - - super().__init__(child, self._collect_references(grouping_cols + initial_state_grouping_cols)) - # raise Exception(f"collect references: {self._collect_references(grouping_cols + initial_state_grouping_cols)}") + if initial_state_plan is not None: + assert isinstance(initial_state_grouping_cols, list)\ + and all(isinstance(c, Column) for c in initial_state_grouping_cols) + super().__init__(child, self._collect_references(grouping_cols + initial_state_grouping_cols)) + else: + super().__init__(child, self._collect_references(grouping_cols)) self._grouping_cols = grouping_cols self._output_schema = output_schema self._output_mode = output_mode self._time_mode = time_mode + self._event_time_col_name = event_time_col_name self._function = function._build_common_inline_user_defined_function(*cols) self._initial_state_plan = initial_state_plan self._initial_state_grouping_cols = initial_state_grouping_cols @@ -2591,10 +2594,11 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: [c.to_plan(session) for c in self._initial_state_grouping_cols] ) - # wrap transformWithStateInPandasUdf in a function plan.transform_with_state_in_pandas.output_schema = self._output_schema plan.transform_with_state_in_pandas.output_mode = self._output_mode plan.transform_with_state_in_pandas.time_mode = self._time_mode + plan.transform_with_state_in_pandas.event_time_col_name = self._event_time_col_name + # wrap transformWithStateInPandasUdf in a function plan.transform_with_state_in_pandas.transform_with_state_udf.CopyFrom( self._function.to_plan_udf(session)) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 88a897aa32d5b..43b9fb3136a62 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -67,6 +67,7 @@ def conf(cls): ) cfg.set("spark.sql.execution.arrow.transformWithStateInPandas.maxRecordsPerBatch", "2") cfg.set("spark.sql.session.timeZone", "UTC") + cfg.set("spark.sql.streaming.noDataMicroBatches.enabled", "false") return cfg def _prepare_input_data(self, input_path, col1, col2): @@ -105,12 +106,6 @@ def _prepare_input_data_with_3_cols(self, input_path, col1, col2, col3): for e1, e2, e3 in zip(col1, col2, col3): fw.write(f"{e1},{e2},{e3}\n") - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - def _check_query_end_exception(self, error): - error_msg = str(error) - return "Checks passed, ending the query for processing time mode" in error_msg - def build_test_df_with_3_cols(self, input_path): df = self.spark.readStream.format("text").option("maxFilesPerTrigger", 1).load(input_path) df_split = df.withColumn("split_values", split(df["value"], ",")) @@ -121,7 +116,6 @@ def build_test_df_with_3_cols(self, input_path): ) return df_final - """ def _test_transform_with_state_in_pandas_basic( self, stateful_processor, @@ -277,22 +271,14 @@ def check_results(batch_df, _): # test list state with ttl has the same behavior as list state when state doesn't expire. def test_transform_with_state_in_pandas_list_state_large_ttl(self): def check_results(batch_df, batch_id): - if batch_id == 0: - assert set(batch_df.sort("id").collect()) == { - Row(id="0", countAsString="2"), - Row(id="1", countAsString="2"), - } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } - try: - self._test_transform_with_state_in_pandas_basic( - ListStateLargeTTLProcessor(), check_results, True, "processingTime" - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_basic( + ListStateLargeTTLProcessor(), check_results, True, "processingTime" + ) def test_transform_with_state_in_pandas_map_state(self): def check_results(batch_df, _): @@ -306,22 +292,14 @@ def check_results(batch_df, _): # test map state with ttl has the same behavior as map state when state doesn't expire. def test_transform_with_state_in_pandas_map_state_large_ttl(self): def check_results(batch_df, batch_id): - if batch_id == 0: - assert set(batch_df.sort("id").collect()) == { - Row(id="0", countAsString="2"), - Row(id="1", countAsString="2"), - } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } - try: - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), check_results, True, "processingTime" - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), check_results, True, "processingTime" + ) # test value state with ttl has the same behavior as value state when # state doesn't expire. @@ -332,21 +310,15 @@ def check_results(batch_df, batch_id): Row(id="0", countAsString="2"), Row(id="1", countAsString="2"), } - elif batch_id == 1: + else: assert set(batch_df.sort("id").collect()) == { Row(id="0", countAsString="3"), Row(id="1", countAsString="2"), } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - try: - self._test_transform_with_state_in_pandas_basic( - SimpleTTLStatefulProcessor(), check_results, False, "processingTime" - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + + self._test_transform_with_state_in_pandas_basic( + SimpleTTLStatefulProcessor(), check_results, False, "processingTime" + ) # TODO SPARK-50908 holistic fix for TTL suite @unittest.skip("test is flaky and it is only a timing issue, skipping until we can resolve") @@ -401,10 +373,7 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=3), ], ) - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") + if batch_id == 0 or batch_id == 1: time.sleep(4) @@ -441,8 +410,6 @@ def check_results(batch_df, batch_id): q.stop() q.awaitTermination() self.assertTrue(q.exception() is None) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) finally: input_dir.cleanup() @@ -490,6 +457,7 @@ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, ch self.assertTrue(q.exception() is None) # TODO fix later + @unittest.skip("fix later") def test_transform_with_state_in_pandas_proc_timer(self): # helper function to check expired timestamp is smaller than current processing time def check_timestamp(batch_df): @@ -526,7 +494,7 @@ def check_results(batch_df, batch_id): ).first()["timeValues"] # check_timestamp(batch_df) - elif batch_id == 2: + else: assert set(batch_df.sort("id").select("id", "countAsString").collect()) == { Row(id="0", countAsString="3"), Row(id="0", countAsString="-1"), @@ -540,17 +508,9 @@ def check_results(batch_df, batch_id): ).first()["timeValues"] # assert current_batch_expired_timestamp > self.first_expired_timestamp - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - - try: - self._test_transform_with_state_in_pandas_proc_timer( - ProcTimeStatefulProcessor(), check_results - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_proc_timer( + ProcTimeStatefulProcessor(), check_results + ) def _test_transform_with_state_in_pandas_event_time(self, stateful_processor, check_results): import pyspark.sql.functions as f @@ -629,7 +589,7 @@ def check_results(batch_df, batch_id): Row(id="a", timestamp="4"), Row(id="a-expired", timestamp="10000"), } - elif batch_id == 2: + else: # watermark for late event = 10 # watermark for eviction = 10 (unchanged as 4 < 10) # timer is registered with expiration time = 10, hence expired at the same batch @@ -637,18 +597,10 @@ def check_results(batch_df, batch_id): Row(id="a", timestamp="15"), Row(id="a-expired", timestamp="10000"), } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - try: - self._test_transform_with_state_in_pandas_event_time( - EventTimeStatefulProcessor(), check_results - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) - """ + self._test_transform_with_state_in_pandas_event_time( + EventTimeStatefulProcessor(), check_results + ) def _test_transform_with_state_init_state_in_pandas( self, @@ -727,7 +679,6 @@ def check_results(batch_df, batch_id): SimpleStatefulProcessorWithInitialState(), check_results ) - """ def _test_transform_with_state_non_contiguous_grouping_cols( self, stateful_processor, check_results, initial_state=None ): @@ -773,35 +724,22 @@ def _test_transform_with_state_non_contiguous_grouping_cols( def test_transform_with_state_non_contiguous_grouping_cols(self): def check_results(batch_df, batch_id): - if batch_id == 0: - assert set(batch_df.collect()) == { - Row(id1="0", id2="1", value=str(123 + 46)), - Row(id1="1", id2="2", value=str(146 + 346)), - } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") + assert set(batch_df.collect()) == { + Row(id1="0", id2="1", value=str(123 + 46)), + Row(id1="1", id2="2", value=str(146 + 346)), + } - try: - self._test_transform_with_state_non_contiguous_grouping_cols( - SimpleStatefulProcessorWithInitialState(), check_results - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_non_contiguous_grouping_cols( + SimpleStatefulProcessorWithInitialState(), check_results + ) def test_transform_with_state_non_contiguous_grouping_cols_with_init_state(self): def check_results(batch_df, batch_id): - if batch_id == 0: - # initial state for key (0, 1) is processed - assert set(batch_df.collect()) == { - Row(id1="0", id2="1", value=str(789 + 123 + 46)), - Row(id1="1", id2="2", value=str(146 + 346)), - } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") + # initial state for key (0, 1) is processed + assert set(batch_df.collect()) == { + Row(id1="0", id2="1", value=str(789 + 123 + 46)), + Row(id1="1", id2="2", value=str(146 + 346)), + } # grouping key of initial state is also not starting from the beginning of attributes data = [(789, "0", "1"), (987, "3", "2")] @@ -809,14 +747,10 @@ def check_results(batch_df, batch_id): data, "initVal int, id1 string, id2 string" ).groupBy("id1", "id2") - try: - self._test_transform_with_state_non_contiguous_grouping_cols( - SimpleStatefulProcessorWithInitialState(), check_results, initial_state - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_non_contiguous_grouping_cols( + SimpleStatefulProcessorWithInitialState(), check_results, initial_state + ) - @unittest.skip("chaining of ops not supported yet") def _test_transform_with_state_in_pandas_chaining_ops( self, stateful_processor, check_results, timeMode="None", grouping_cols=["outputTimestamp"] ): @@ -867,6 +801,7 @@ def _test_transform_with_state_in_pandas_chaining_ops( q.processAllAvailable() q.awaitTermination(10) + @unittest.skip("chaining of ops not supported yet") def test_transform_with_state_in_pandas_chaining_ops(self): def check_results(batch_df, batch_id): import datetime @@ -916,24 +851,17 @@ def check_results(batch_df, batch_id): Row(id="0", value=str(789 + 123 + 46)), Row(id="1", value=str(146 + 346)), } - elif batch_id == 1: + else: # handleInitialState is only processed in the first batch, # no more timer is registered so no more expired timers assert set(batch_df.sort("id").collect()) == { Row(id="0", value=str(789 + 123 + 46 + 67)), Row(id="3", value=str(987 + 12)), } - else: - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - try: - self._test_transform_with_state_init_state_in_pandas( - StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_init_state_in_pandas( + StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" + ) def test_transform_with_state_in_pandas_batch_query(self): data = [("0", 123), ("0", 46), ("1", 146), ("1", 346)] @@ -1091,21 +1019,14 @@ def check_results(batch_df, batch_id): ) assert list_state_df.isEmpty() - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - - try: - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=None, - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) # run the same test suite again but with no-op initial state # TWS with initial state is using a different python runner @@ -1113,17 +1034,15 @@ def check_results(batch_df, batch_id): initial_state = self.spark.createDataFrame(init_data, "id string, temperature int").groupBy( "id" ) - try: - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=initial_state, - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + + self._test_transform_with_state_in_pandas_basic( + MapStateLargeTTLProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=initial_state, + ) # This test covers multiple list state variables and flatten option def test_transform_with_list_state_metadata(self): @@ -1192,21 +1111,14 @@ def check_results(batch_df, batch_id): Row(groupingKey="1", valueSortedList=[20, 20, 120, 120, 222]), ] - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - - try: - self._test_transform_with_state_in_pandas_basic( - ListStateProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=None, - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_basic( + ListStateProcessor(), + check_results, + True, + "processingTime", + checkpoint_path=checkpoint_path, + initial_state=None, + ) # This test covers value state variable and read change feed, # snapshotStartBatchId related options @@ -1286,23 +1198,16 @@ def check_results(batch_df, batch_id): .collect() ) - # TODO SPARK-50180 This is a hack to exit the query when all assertions are passed - # for processing time mode - raise Exception("Checks passed, ending the query for processing time mode") - with self.sql_conf( {"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled": "true"} ): - try: - self._test_transform_with_state_in_pandas_basic( - SimpleStatefulProcessor(), - check_results, - False, - "processingTime", - checkpoint_path=checkpoint_path, - ) - except Exception as e: - self.assertTrue(self._check_query_end_exception(e)) + self._test_transform_with_state_in_pandas_basic( + SimpleStatefulProcessor(), + check_results, + False, + "processingTime", + checkpoint_path=checkpoint_path, + ) def test_transform_with_state_restart_with_multiple_rows_init_state(self): def check_results(batch_df, _): @@ -1529,7 +1434,7 @@ def check_exception(error): df, check_exception=check_exception, ) - """ + class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index a5a57c8d82f1e..f084318a2967d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1054,7 +1054,7 @@ class SparkConnectPlanner( outputSchema, rel.getOutputMode, rel.getTimeMode, - initialStateDs, "") + initialStateDs, rel.getEventTimeColName) .logicalPlan } else { Dataset @@ -1065,7 +1065,7 @@ class SparkConnectPlanner( outputSchema, rel.getOutputMode, rel.getTimeMode, - null, "") + null, rel.getEventTimeColName) .logicalPlan } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala index c9925749786f3..14593c1d59a90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala @@ -348,16 +348,12 @@ class RelationalGroupedDataset protected[sql]( val leftGroupingNamedExpressions = groupingExprs.map { case ne: NamedExpression => ne - case other => - throw new Exception(s"I am here, attaching alias") - Alias(other, other.toString)() + case other => Alias(other, other.toString)() } val rightGroupingNamedExpressions = r.groupingExprs.map { case ne: NamedExpression => ne - case other => - throw new Exception(s"I am here, attaching alias") - Alias(other, other.toString)() + case other => Alias(other, other.toString)() } val leftChild = df.logicalPlan @@ -367,13 +363,6 @@ class RelationalGroupedDataset protected[sql]( Project(leftGroupingNamedExpressions ++ leftChild.output, leftChild)).analyzed val right = r.df.sparkSession.sessionState.executePlan( Project(rightGroupingNamedExpressions ++ rightChild.output, rightChild)).analyzed - /* - throw new Exception(f"initial state grouping expr: ${r.groupingExprs}, " + - f"initGroupingAttrs: $rightGroupingNamedExpressions, df grouping expr: ${groupingExprs}, " + - f"df grouping attrs: ${leftGroupingNamedExpressions}") - - throw new Exception(f"rightChild plan: ${rightChild}," + - f" rightGroupingNamedExpressions: $rightGroupingNamedExpressions, right: $right") */ val output = toAttributes(expr.dataType.asInstanceOf[StructType]) val plan = FlatMapCoGroupsInPandas( @@ -490,13 +479,9 @@ class RelationalGroupedDataset protected[sql]( timeModeStr: String, initialState: RelationalGroupedDataset, eventTimeColumnName: String): DataFrame = { - def exprToAttr(expr: Seq[Expression]) = { - expr.map { + def exprToAttr(expr: Seq[Expression]) = expr.map { case ne: NamedExpression => ne - case other => - println(s"I am here, attaching alias") - Alias(other, other.toString)() - } + case other => Alias(other, other.toString)() } val groupingAttrs = exprToAttr(groupingExprs) From d8806842bada7e735697e3493545a2bfdb5cd7ff Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 27 Jan 2025 18:11:24 -0800 Subject: [PATCH 07/20] fix all tests except for 2 --- python/pyspark/sql/connect/group.py | 1 - python/pyspark/sql/pandas/group_ops.py | 3 - .../tests/pandas/test_pandas_cogrouped_map.py | 4 -- .../test_pandas_transform_with_state.py | 65 +++++++------------ .../logical/pythonLogicalOperators.scala | 1 - .../classic/RelationalGroupedDataset.scala | 4 +- 6 files changed, 27 insertions(+), 51 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 37b6bff09dc3b..d7b1f46250126 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -395,7 +395,6 @@ def transformWithStateInPandas( initial_state_plan = initialState._df._plan initial_state_grouping_cols = initialState._grouping_cols - # TODO add a string struct type test output_schema: str = ( outputStructType.json() if isinstance(outputStructType, StructType) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 45171966afaf1..5a03a7b98ff68 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -492,9 +492,6 @@ def transformWithStateInPandas( if isinstance(outputStructType, str): outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) - if isinstance(outputStructType, str): - outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) - df = self._df udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py index 55f3ed267e583..1f9532352679a 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_cogrouped_map.py @@ -71,7 +71,6 @@ def data2(self): .drop("ks") ) - """ def test_simple(self): self._test_merge(self.data1, self.data2) @@ -357,7 +356,6 @@ def check_wrong_args(self): errorClass=ValueError, error_message_regex="Invalid function", ) - """ def test_case_insensitive_grouping_column(self): # SPARK-31915: case-insensitive grouping column should work. @@ -381,7 +379,6 @@ def test_case_insensitive_grouping_column(self): ) self.assertEqual(row.asDict(), Row(column=2, value=2).asDict()) - """ def test_self_join(self): # SPARK-34319: self-join with FlatMapCoGroupsInPandas df = self.spark.createDataFrame([(1, 1)], ("column", "value")) @@ -481,7 +478,6 @@ def summarize(left, right): "| 2| 3| 1| 2|\n" "+---------+------------+----------+-------------+\n", ) - """ @staticmethod def _test_with_key(left, right, isLeft): diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 43b9fb3136a62..88ab0f6da673d 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -116,6 +116,7 @@ def build_test_df_with_3_cols(self, input_path): ) return df_final + """ def _test_transform_with_state_in_pandas_basic( self, stateful_processor, @@ -412,6 +413,7 @@ def check_results(batch_df, batch_id): self.assertTrue(q.exception() is None) finally: input_dir.cleanup() + """ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, check_results): input_path = tempfile.mkdtemp() @@ -459,23 +461,23 @@ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, ch # TODO fix later @unittest.skip("fix later") def test_transform_with_state_in_pandas_proc_timer(self): - # helper function to check expired timestamp is smaller than current processing time - def check_timestamp(batch_df): - expired_df = ( - batch_df.filter(batch_df["countAsString"] == "-1") - .select("id", "timeValues") - .withColumnRenamed("timeValues", "expiredTimestamp") - ) - count_df = ( - batch_df.filter(batch_df["countAsString"] != "-1") - .select("id", "timeValues") - .withColumnRenamed("timeValues", "countStateTimestamp") - ) - joined_df = expired_df.join(count_df, on="id") - for row in joined_df.collect(): - assert row["expiredTimestamp"] < row["countStateTimestamp"] - def check_results(batch_df, batch_id): + # helper function to check expired timestamp is smaller than current processing time + def check_timestamp(batch_df): + expired_df = ( + batch_df.filter(batch_df["countAsString"] == "-1") + .select("id", "timeValues") + .withColumnRenamed("timeValues", "expiredTimestamp") + ) + count_df = ( + batch_df.filter(batch_df["countAsString"] != "-1") + .select("id", "timeValues") + .withColumnRenamed("timeValues", "countStateTimestamp") + ) + joined_df = expired_df.join(count_df, on="id") + for row in joined_df.collect(): + assert row["expiredTimestamp"] < row["countStateTimestamp"] + if batch_id == 0: assert set(batch_df.sort("id").select("id", "countAsString").collect()) == { Row(id="0", countAsString="1"), @@ -492,7 +494,7 @@ def check_results(batch_df, batch_id): self.first_expired_timestamp = batch_df.filter( batch_df["countAsString"] == -1 ).first()["timeValues"] - # check_timestamp(batch_df) + check_timestamp(batch_df) else: assert set(batch_df.sort("id").select("id", "countAsString").collect()) == { @@ -512,6 +514,7 @@ def check_results(batch_df, batch_id): ProcTimeStatefulProcessor(), check_results ) + """ def _test_transform_with_state_in_pandas_event_time(self, stateful_processor, check_results): import pyspark.sql.functions as f @@ -623,12 +626,7 @@ def _test_transform_with_state_init_state_in_pandas( df = self._build_test_df(input_path) self.assertTrue(df.isStreaming) - output_schema = StructType( - [ - StructField("id", StringType(), True), - StructField("value", StringType(), True), - ] - ) + output_schema = "id string, value str" if initial_state is None: data = [("0", 789), ("3", 987)] @@ -801,7 +799,6 @@ def _test_transform_with_state_in_pandas_chaining_ops( q.processAllAvailable() q.awaitTermination(10) - @unittest.skip("chaining of ops not supported yet") def test_transform_with_state_in_pandas_chaining_ops(self): def check_results(batch_df, batch_id): import datetime @@ -1273,6 +1270,7 @@ def test_transform_with_state_with_timers_single_partition(self): self.test_transform_with_state_in_pandas_event_time() self.test_transform_with_state_in_pandas_proc_timer() self.test_transform_with_state_restart_with_multiple_rows_init_state() + """ def _run_evolution_test( self, processor, checkpoint_dir, check_results, df, check_exception=None @@ -1319,23 +1317,20 @@ def _run_evolution_test( self.assertTrue(check_exception(e)) def test_schema_evolution_scenarios(self): - # Test various schema evolution scenarios + """Test various schema evolution scenarios""" with self.sql_conf({"spark.sql.streaming.stateStore.encodingFormat": "avro"}): with tempfile.TemporaryDirectory() as checkpoint_dir: # Test 1: Basic state input_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) - df = self._build_test_df(input_path) def check_basic_state(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 0 # First ID from test data assert result.value["name"] == "name-0" - self._run_evolution_test(BasicProcessor(), checkpoint_dir, check_basic_state, df) - self._prepare_test_resource2(input_path) # Test 2: Add fields @@ -1346,7 +1341,6 @@ def check_add_fields(batch_df, batch_id): assert result.value["count"] is None assert result.value["active"] is None assert result.value["score"] is None - self._run_evolution_test(AddFieldsProcessor(), checkpoint_dir, check_add_fields, df) self._prepare_test_resource3(input_path) @@ -1354,8 +1348,7 @@ def check_add_fields(batch_df, batch_id): def check_remove_fields(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 0 # First ID from test data - assert result.value["name"] == "name-00" - + assert result.value["name"] == "name-00", f"batch id: {batch_id}, real df: {batch_df.collect()}" self._run_evolution_test( RemoveFieldsProcessor(), checkpoint_dir, check_remove_fields, df ) @@ -1366,7 +1359,6 @@ def check_reorder_fields(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["name"] == "name-00" assert result.value["id"] == 0 - self._run_evolution_test( ReorderedFieldsProcessor(), checkpoint_dir, check_reorder_fields, df ) @@ -1377,7 +1369,6 @@ def check_upcast(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 1 assert result.value["name"] == "name-0" - self._run_evolution_test(UpcastProcessor(), checkpoint_dir, check_upcast, df) # This test case verifies that an exception is thrown when downcasting, which violates @@ -1387,25 +1378,20 @@ def test_schema_evolution_fails(self): with tempfile.TemporaryDirectory() as checkpoint_dir: input_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) - df = self._build_test_df(input_path) def check_add_fields(batch_df, batch_id): results = batch_df.collect() assert results[0].value["count"] == 100 assert results[0].value["active"] - self._run_evolution_test(AddFieldsProcessor(), checkpoint_dir, check_add_fields, df) - self._prepare_test_resource2(input_path) def check_upcast(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["name"] == "name-0" - # Long self._run_evolution_test(UpcastProcessor(), checkpoint_dir, check_upcast, df) - self._prepare_test_resource3(input_path) def check_basic_state(batch_df, batch_id): @@ -1415,17 +1401,14 @@ def check_basic_state(batch_df, batch_id): def check_exception(error): from pyspark.errors.exceptions.captured import StreamingQueryException - if not isinstance(error, StreamingQueryException): return False - error_msg = str(error) return ( "[STREAM_FAILED]" in error_msg and "[STATE_STORE_INVALID_VALUE_SCHEMA_EVOLUTION]" in error_msg and "Schema evolution is not possible" in error_msg ) - # Int self._run_evolution_test( BasicProcessor(), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 3ab5accee2a25..2f9bf2b52190a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -240,7 +240,6 @@ case class FlatMapCoGroupsInArrow( override val producedAttributes = AttributeSet(output) override lazy val references: AttributeSet = AttributeSet(leftAttributes ++ rightAttributes ++ functionExpr.references) -- producedAttributes - throw new Exception(s"what is the references here: ${references.toSeq}") def leftAttributes: Seq[Attribute] = left.output.take(leftGroupingLen) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala index 14593c1d59a90..082292145e858 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/RelationalGroupedDataset.scala @@ -479,9 +479,11 @@ class RelationalGroupedDataset protected[sql]( timeModeStr: String, initialState: RelationalGroupedDataset, eventTimeColumnName: String): DataFrame = { - def exprToAttr(expr: Seq[Expression]) = expr.map { + def exprToAttr(expr: Seq[Expression]): Seq[Attribute] = { + expr.map { case ne: NamedExpression => ne case other => Alias(other, other.toString)() + }.map(_.toAttribute) } val groupingAttrs = exprToAttr(groupingExprs) From 07cbcf4dfee150ebe412cc5b62f08735b177e89d Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 27 Jan 2025 18:13:19 -0800 Subject: [PATCH 08/20] update proto files --- .../sql/connect/proto/relations_pb2.py | 40 +++++++++---------- .../sql/connect/proto/relations_pb2.pyi | 14 +------ .../protobuf/spark/connect/relations.proto | 4 +- 3 files changed, 24 insertions(+), 34 deletions(-) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 94b0717b190ee..fc5c923434072 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x8d\x1e\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12o\n\x1etransform_with_state_in_pandas\x18- \x01(\x0b\x32).spark.connect.TransformWithStateInPandasH\x00R\x1atransformWithStateInPandas\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xcc\x04\n\x1aTransformWithStateInPandas\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\x12g\n\x18transform_with_state_udf\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x15transformWithStateUdf\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12\x1f\n\x0boutput_mode\x18\x05 \x01(\tR\noutputMode\x12\x1b\n\ttime_mode\x18\x06 \x01(\tR\x08timeMode\x12<\n\rinitial_input\x18\x07 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x08 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12\x32\n\x13\x65vent_time_col_name\x18\t \x01(\tH\x00R\x10\x65ventTimeColName\x88\x01\x01\x42\x16\n\x14_event_time_col_name"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto\x1a\x1dspark/connect/ml_common.proto"\x8d\x1e\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12o\n\x1etransform_with_state_in_pandas\x18- \x01(\x0b\x32).spark.connect.TransformWithStateInPandasH\x00R\x1atransformWithStateInPandas\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xaf\x04\n\x1aTransformWithStateInPandas\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\x12g\n\x18transform_with_state_udf\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x15transformWithStateUdf\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12\x1f\n\x0boutput_mode\x18\x05 \x01(\tR\noutputMode\x12\x1b\n\ttime_mode\x18\x06 \x01(\tR\x08timeMode\x12<\n\rinitial_input\x18\x07 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x08 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12-\n\x13\x65vent_time_col_name\x18\t \x01(\tR\x10\x65ventTimeColName"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -225,25 +225,25 @@ _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15261 _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15618 _globals["_TRANSFORMWITHSTATEINPANDAS"]._serialized_start = 15621 - _globals["_TRANSFORMWITHSTATEINPANDAS"]._serialized_end = 16209 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 16212 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16456 - _globals["_PYTHONUDTF"]._serialized_start = 16459 - _globals["_PYTHONUDTF"]._serialized_end = 16636 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16639 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16790 - _globals["_PYTHONDATASOURCE"]._serialized_start = 16792 - _globals["_PYTHONDATASOURCE"]._serialized_end = 16867 - _globals["_COLLECTMETRICS"]._serialized_start = 16870 - _globals["_COLLECTMETRICS"]._serialized_end = 17006 - _globals["_PARSE"]._serialized_start = 17009 - _globals["_PARSE"]._serialized_end = 17397 + _globals["_TRANSFORMWITHSTATEINPANDAS"]._serialized_end = 16180 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 16183 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16427 + _globals["_PYTHONUDTF"]._serialized_start = 16430 + _globals["_PYTHONUDTF"]._serialized_end = 16607 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16610 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16761 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16763 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16838 + _globals["_COLLECTMETRICS"]._serialized_start = 16841 + _globals["_COLLECTMETRICS"]._serialized_end = 16977 + _globals["_PARSE"]._serialized_start = 16980 + _globals["_PARSE"]._serialized_end = 17368 _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5861 _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5919 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17298 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17386 - _globals["_ASOFJOIN"]._serialized_start = 17400 - _globals["_ASOFJOIN"]._serialized_end = 17875 - _globals["_LATERALJOIN"]._serialized_start = 17878 - _globals["_LATERALJOIN"]._serialized_end = 18108 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 17269 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17357 + _globals["_ASOFJOIN"]._serialized_start = 17371 + _globals["_ASOFJOIN"]._serialized_end = 17846 + _globals["_LATERALJOIN"]._serialized_start = 17849 + _globals["_LATERALJOIN"]._serialized_end = 18079 # @@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 cbf339ada2959..525a831e1abe1 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -3983,7 +3983,7 @@ class TransformWithStateInPandas(google.protobuf.message.Message): ]: """(Optional) Expressions for grouping keys of the initial state input relation.""" event_time_col_name: builtins.str - """(Optional) Event time column name""" + """(Required) Event time column name. Default to be empty string.""" def __init__( self, *, @@ -4002,15 +4002,11 @@ class TransformWithStateInPandas(google.protobuf.message.Message): pyspark.sql.connect.proto.expressions_pb2.Expression ] | None = ..., - event_time_col_name: builtins.str | None = ..., + event_time_col_name: builtins.str = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal[ - "_event_time_col_name", - b"_event_time_col_name", - "event_time_col_name", - b"event_time_col_name", "initial_input", b"initial_input", "input", @@ -4022,8 +4018,6 @@ class TransformWithStateInPandas(google.protobuf.message.Message): def ClearField( self, field_name: typing_extensions.Literal[ - "_event_time_col_name", - b"_event_time_col_name", "event_time_col_name", b"event_time_col_name", "grouping_expressions", @@ -4044,10 +4038,6 @@ class TransformWithStateInPandas(google.protobuf.message.Message): b"transform_with_state_udf", ], ) -> None: ... - def WhichOneof( - self, - oneof_group: typing_extensions.Literal["_event_time_col_name", b"_event_time_col_name"], - ) -> typing_extensions.Literal["event_time_col_name"] | None: ... global___TransformWithStateInPandas = TransformWithStateInPandas diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index e17a17cdd05c5..840de0dfcf100 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -1105,8 +1105,8 @@ message TransformWithStateInPandas { // (Optional) Expressions for grouping keys of the initial state input relation. repeated Expression initial_grouping_expressions = 8; - // (Optional) Event time column name - optional string event_time_col_name = 9; + // (Required) Event time column name. Default to be empty string. + string event_time_col_name = 9; } message CommonInlineUserDefinedTableFunction { From e93313214290434315fa568489689718a564c2f5 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 29 Jan 2025 15:23:19 -0800 Subject: [PATCH 09/20] fix all suites, lint --- dev/sparktestsupport/modules.py | 2 +- python/pyspark/sql/connect/plan.py | 16 +- .../sql/streaming/stateful_processor_util.py | 55 +++---- .../test_pandas_transform_with_state.py | 150 +++++++----------- .../connect/planner/SparkConnectPlanner.scala | 10 +- 5 files changed, 99 insertions(+), 134 deletions(-) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 6b935c4378764..cece8fd415faa 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1096,7 +1096,7 @@ def __hash__(self): "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_scalar", "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_grouped_agg", "pyspark.sql.tests.connect.pandas.test_parity_pandas_udf_window", - "pyspark.sql.tests.connect.pandas.test_parity_pandas_transform_with_state" + "pyspark.sql.tests.connect.pandas.test_parity_pandas_transform_with_state", ], excluded_python_implementations=[ "PyPy" # Skip these tests under PyPy since they require numpy, pandas, and pyarrow and diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 58553facfb09a..1fec45088e477 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2565,9 +2565,12 @@ def __init__( ): assert isinstance(grouping_cols, list) and all(isinstance(c, Column) for c in grouping_cols) if initial_state_plan is not None: - assert isinstance(initial_state_grouping_cols, list)\ - and all(isinstance(c, Column) for c in initial_state_grouping_cols) - super().__init__(child, self._collect_references(grouping_cols + initial_state_grouping_cols)) + assert isinstance(initial_state_grouping_cols, list) and all( + isinstance(c, Column) for c in initial_state_grouping_cols + ) + super().__init__( + child, self._collect_references(grouping_cols + initial_state_grouping_cols) + ) else: super().__init__(child, self._collect_references(grouping_cols)) self._grouping_cols = grouping_cols @@ -2589,7 +2592,9 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: # fill in initial state related fields if self._initial_state_plan is not None: self._initial_state_plan = cast(LogicalPlan, self._initial_state_plan) - plan.transform_with_state_in_pandas.initial_input.CopyFrom(self._initial_state_plan.plan(session)) + plan.transform_with_state_in_pandas.initial_input.CopyFrom( + self._initial_state_plan.plan(session) + ) plan.transform_with_state_in_pandas.initial_grouping_expressions.extend( [c.to_plan(session) for c in self._initial_state_grouping_cols] ) @@ -2600,7 +2605,8 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: plan.transform_with_state_in_pandas.event_time_col_name = self._event_time_col_name # wrap transformWithStateInPandasUdf in a function plan.transform_with_state_in_pandas.transform_with_state_udf.CopyFrom( - self._function.to_plan_udf(session)) + self._function.to_plan_udf(session) + ) return self._with_relations(plan, session) diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index a8e91337eba91..df547145eef8a 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -29,6 +29,7 @@ TimerValues, ) from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle + if TYPE_CHECKING: from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode @@ -45,17 +46,16 @@ class TransformWithStateInPandasFuncMode(Enum): class TransformWithStateInPandasUdfUtils: - def __init__(self, stateful_processor: StatefulProcessor, time_mode: str): self._stateful_processor = stateful_processor self._time_mode = time_mode def transformWithStateUDF( - self, - stateful_processor_api_client: StatefulProcessorApiClient, - mode: TransformWithStateInPandasFuncMode, - key: Any, - input_rows: Iterator["PandasDataFrameLike"], + self, + stateful_processor_api_client: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + input_rows: Iterator["PandasDataFrameLike"], ) -> Iterator["PandasDataFrameLike"]: if mode == TransformWithStateInPandasFuncMode.PRE_INIT: return self._handle_pre_init(stateful_processor_api_client) @@ -64,9 +64,7 @@ def transformWithStateUDF( if stateful_processor_api_client.handle_state == StatefulProcessorHandleState.CREATED: self._stateful_processor.init(handle) - stateful_processor_api_client.set_handle_state( - StatefulProcessorHandleState.INITIALIZED - ) + stateful_processor_api_client.set_handle_state(StatefulProcessorHandleState.INITIALIZED) if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: stateful_processor_api_client.set_handle_state( @@ -88,12 +86,12 @@ def transformWithStateUDF( return result def transformWithStateWithInitStateUDF( - self, - stateful_processor_api_client: StatefulProcessorApiClient, - mode: TransformWithStateInPandasFuncMode, - key: Any, - input_rows: Iterator["PandasDataFrameLike"], - initial_states: Optional[Iterator["PandasDataFrameLike"]] = None, + self, + stateful_processor_api_client: StatefulProcessorApiClient, + mode: TransformWithStateInPandasFuncMode, + key: Any, + input_rows: Iterator["PandasDataFrameLike"], + initial_states: Optional[Iterator["PandasDataFrameLike"]] = None, ) -> Iterator["PandasDataFrameLike"]: """ UDF for TWS operator with non-empty initial states. Possible input combinations @@ -114,9 +112,7 @@ def transformWithStateWithInitStateUDF( if stateful_processor_api_client.handle_state == StatefulProcessorHandleState.CREATED: self._stateful_processor.init(handle) - stateful_processor_api_client.set_handle_state( - StatefulProcessorHandleState.INITIALIZED - ) + stateful_processor_api_client.set_handle_state(StatefulProcessorHandleState.INITIALIZED) if mode == TransformWithStateInPandasFuncMode.PROCESS_TIMER: stateful_processor_api_client.set_handle_state( @@ -161,8 +157,7 @@ def transformWithStateWithInitStateUDF( return result def _handle_pre_init( - self, - stateful_processor_api_client: StatefulProcessorApiClient + self, stateful_processor_api_client: StatefulProcessorApiClient ) -> Iterator["PandasDataFrameLike"]: # Driver handle is different from the handle used on executors; # On JVM side, we will use `DriverStatefulProcessorHandleImpl` for driver handle which @@ -180,10 +175,10 @@ def _handle_pre_init( return iter([]) def _handle_data_rows( - self, - stateful_processor_api_client: StatefulProcessorApiClient, - key: Any, - input_rows: Optional[Iterator["PandasDataFrameLike"]] = None, + self, + stateful_processor_api_client: StatefulProcessorApiClient, + key: Any, + input_rows: Optional[Iterator["PandasDataFrameLike"]] = None, ) -> Iterator["PandasDataFrameLike"]: stateful_processor_api_client.set_implicit_key(key) @@ -201,8 +196,8 @@ def _handle_data_rows( return iter([]) def _handle_expired_timers( - self, - stateful_processor_api_client: StatefulProcessorApiClient, + self, + stateful_processor_api_client: StatefulProcessorApiClient, ) -> Iterator["PandasDataFrameLike"]: batch_timestamp, watermark_timestamp = stateful_processor_api_client.get_timestamps( self._time_mode @@ -224,9 +219,9 @@ def _handle_expired_timers( for key_obj, expiry_timestamp in expiry_list: stateful_processor_api_client.set_implicit_key(key_obj) for pd in self._stateful_processor.handleExpiredTimer( - key=key_obj, - timerValues=TimerValues(batch_timestamp, watermark_timestamp), - expiredTimerInfo=ExpiredTimerInfo(expiry_timestamp), + key=key_obj, + timerValues=TimerValues(batch_timestamp, watermark_timestamp), + expiredTimerInfo=ExpiredTimerInfo(expiry_timestamp), ): yield pd - stateful_processor_api_client.delete_timer(expiry_timestamp) \ No newline at end of file + stateful_processor_api_client.delete_timer(expiry_timestamp) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 88ab0f6da673d..03797e0e6712f 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -59,7 +59,9 @@ class TransformWithStateInPandasTestsMixin: @classmethod def conf(cls): - cfg = SparkConf(loadDefaults=False) # Avoid loading default configs + cfg = SparkConf( + loadDefaults=False + ) # Avoid loading default configs so that connect suites can run cfg.set("spark.sql.shuffle.partitions", "5") cfg.set( "spark.sql.streaming.stateStore.providerClass", @@ -116,7 +118,6 @@ def build_test_df_with_3_cols(self, input_path): ) return df_final - """ def _test_transform_with_state_in_pandas_basic( self, stateful_processor, @@ -321,8 +322,6 @@ def check_results(batch_df, batch_id): SimpleTTLStatefulProcessor(), check_results, False, "processingTime" ) - # TODO SPARK-50908 holistic fix for TTL suite - @unittest.skip("test is flaky and it is only a timing issue, skipping until we can resolve") def test_value_state_ttl_expiration(self): def check_results(batch_df, batch_id): if batch_id == 0: @@ -339,44 +338,24 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=1), ], ) - elif batch_id == 1: - assertDataFrameEqual( - batch_df, - [ - Row(id="ttl-count-0", count=2), - Row(id="count-0", count=2), - Row(id="ttl-list-state-count-0", count=3), - Row(id="ttl-map-state-count-0", count=2), - Row(id="ttl-count-1", count=2), - Row(id="count-1", count=2), - Row(id="ttl-list-state-count-1", count=3), - Row(id="ttl-map-state-count-1", count=2), - ], - ) - elif batch_id == 2: - # ttl-count-0 expire and restart from count 0. - # The TTL for value state ttl_count_state gets reset in batch 1 because of the - # update operation and ttl-count-1 keeps the state. - # ttl-list-state-count-0 expire and restart from count 0. - # The TTL for list state ttl_list_state gets reset in batch 1 because of the - # put operation and ttl-list-state-count-1 keeps the state. - # non-ttl state never expires + else: assertDataFrameEqual( batch_df, [ Row(id="ttl-count-0", count=1), - Row(id="count-0", count=3), + Row(id="count-0", count=2), Row(id="ttl-list-state-count-0", count=1), Row(id="ttl-map-state-count-0", count=1), - Row(id="ttl-count-1", count=3), - Row(id="count-1", count=3), - Row(id="ttl-list-state-count-1", count=7), - Row(id="ttl-map-state-count-1", count=3), + Row(id="ttl-count-1", count=1), + Row(id="count-1", count=2), + Row(id="ttl-list-state-count-1", count=1), + Row(id="ttl-map-state-count-1", count=1), ], ) - if batch_id == 0 or batch_id == 1: - time.sleep(4) + if batch_id == 0: + # let ttl state expires + time.sleep(2) input_dir = tempfile.TemporaryDirectory() input_path = input_dir.name @@ -384,7 +363,6 @@ def check_results(batch_df, batch_id): df = self._build_test_df(input_path) self._prepare_input_data(input_path + "/batch1.txt", [1, 0], [0, 0]) self._prepare_input_data(input_path + "/batch2.txt", [1, 0], [0, 0]) - self._prepare_input_data(input_path + "/batch3.txt", [1, 0], [0, 0]) for q in self.spark.streams.active: q.stop() output_schema = StructType( @@ -413,7 +391,6 @@ def check_results(batch_df, batch_id): self.assertTrue(q.exception() is None) finally: input_dir.cleanup() - """ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, check_results): input_path = tempfile.mkdtemp() @@ -458,8 +435,6 @@ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, ch q.awaitTermination(10) self.assertTrue(q.exception() is None) - # TODO fix later - @unittest.skip("fix later") def test_transform_with_state_in_pandas_proc_timer(self): def check_results(batch_df, batch_id): # helper function to check expired timestamp is smaller than current processing time @@ -491,9 +466,6 @@ def check_timestamp(batch_df): Row(id="0", countAsString="-1"), Row(id="1", countAsString="3"), } - self.first_expired_timestamp = batch_df.filter( - batch_df["countAsString"] == -1 - ).first()["timeValues"] check_timestamp(batch_df) else: @@ -502,19 +474,11 @@ def check_timestamp(batch_df): Row(id="0", countAsString="-1"), Row(id="1", countAsString="5"), } - # The expired timestamp in current batch is larger than expiry timestamp in batch 1 - # because this is a new timer registered in batch1 and - # different from the one registered in batch 0 - current_batch_expired_timestamp = batch_df.filter( - batch_df["countAsString"] == -1 - ).first()["timeValues"] - # assert current_batch_expired_timestamp > self.first_expired_timestamp self._test_transform_with_state_in_pandas_proc_timer( ProcTimeStatefulProcessor(), check_results ) - """ def _test_transform_with_state_in_pandas_event_time(self, stateful_processor, check_results): import pyspark.sql.functions as f @@ -1270,11 +1234,8 @@ def test_transform_with_state_with_timers_single_partition(self): self.test_transform_with_state_in_pandas_event_time() self.test_transform_with_state_in_pandas_proc_timer() self.test_transform_with_state_restart_with_multiple_rows_init_state() - """ - def _run_evolution_test( - self, processor, checkpoint_dir, check_results, df, check_exception=None - ): + def _run_evolution_test(self, processor, checkpoint_dir, check_results, df): output_schema = StructType( [ StructField("id", StringType(), True), @@ -1286,35 +1247,25 @@ def _run_evolution_test( for q in self.spark.streams.active: q.stop() - try: - q = ( - df.groupBy("id") - .transformWithStateInPandas( - statefulProcessor=processor, - outputStructType=output_schema, - outputMode="Update", - timeMode="None", - ) - .writeStream.queryName("evolution_test") - .option("checkpointLocation", checkpoint_dir) - .foreachBatch(check_results) - .outputMode("update") - .start() + q = ( + df.groupBy("id") + .transformWithStateInPandas( + statefulProcessor=processor, + outputStructType=output_schema, + outputMode="Update", + timeMode="None", ) + .writeStream.queryName("evolution_test") + .option("checkpointLocation", checkpoint_dir) + .foreachBatch(check_results) + .outputMode("update") + .start() + ) - self.assertEqual(q.name, "evolution_test") - self.assertTrue(q.isActive) - q.processAllAvailable() - q.awaitTermination(10) - - if q.exception() is None: - assert check_exception is None - - except Exception as e: - # If we are expecting an exception, verify it's the right one - if check_exception is None: - raise # Re-raise if we weren't expecting an exception - self.assertTrue(check_exception(e)) + self.assertEqual(q.name, "evolution_test") + self.assertTrue(q.isActive) + q.processAllAvailable() + q.awaitTermination(10) def test_schema_evolution_scenarios(self): """Test various schema evolution scenarios""" @@ -1330,6 +1281,7 @@ def check_basic_state(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 0 # First ID from test data assert result.value["name"] == "name-0" + self._run_evolution_test(BasicProcessor(), checkpoint_dir, check_basic_state, df) self._prepare_test_resource2(input_path) @@ -1341,6 +1293,7 @@ def check_add_fields(batch_df, batch_id): assert result.value["count"] is None assert result.value["active"] is None assert result.value["score"] is None + self._run_evolution_test(AddFieldsProcessor(), checkpoint_dir, check_add_fields, df) self._prepare_test_resource3(input_path) @@ -1348,7 +1301,10 @@ def check_add_fields(batch_df, batch_id): def check_remove_fields(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 0 # First ID from test data - assert result.value["name"] == "name-00", f"batch id: {batch_id}, real df: {batch_df.collect()}" + assert ( + result.value["name"] == "name-00" + ), f"batch id: {batch_id}, real df: {batch_df.collect()}" + self._run_evolution_test( RemoveFieldsProcessor(), checkpoint_dir, check_remove_fields, df ) @@ -1359,6 +1315,7 @@ def check_reorder_fields(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["name"] == "name-00" assert result.value["id"] == 0 + self._run_evolution_test( ReorderedFieldsProcessor(), checkpoint_dir, check_reorder_fields, df ) @@ -1369,6 +1326,7 @@ def check_upcast(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 1 assert result.value["name"] == "name-0" + self._run_evolution_test(UpcastProcessor(), checkpoint_dir, check_upcast, df) # This test case verifies that an exception is thrown when downcasting, which violates @@ -1384,12 +1342,14 @@ def check_add_fields(batch_df, batch_id): results = batch_df.collect() assert results[0].value["count"] == 100 assert results[0].value["active"] + self._run_evolution_test(AddFieldsProcessor(), checkpoint_dir, check_add_fields, df) self._prepare_test_resource2(input_path) def check_upcast(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["name"] == "name-0" + # Long self._run_evolution_test(UpcastProcessor(), checkpoint_dir, check_upcast, df) self._prepare_test_resource3(input_path) @@ -1399,24 +1359,26 @@ def check_basic_state(batch_df, batch_id): assert result.value["id"] == 0 # First ID from test data assert result.value["name"] == "name-0" - def check_exception(error): + # Int + try: + self._run_evolution_test( + BasicProcessor(), + checkpoint_dir, + check_basic_state, + df, + ) + except Exception as e: + # we are expecting an exception, verify it's the right one from pyspark.errors.exceptions.captured import StreamingQueryException - if not isinstance(error, StreamingQueryException): + + if not isinstance(e, StreamingQueryException): return False error_msg = str(error) - return ( + assert ( "[STREAM_FAILED]" in error_msg and "[STATE_STORE_INVALID_VALUE_SCHEMA_EVOLUTION]" in error_msg and "Schema evolution is not possible" in error_msg ) - # Int - self._run_evolution_test( - BasicProcessor(), - checkpoint_dir, - check_basic_state, - df, - check_exception=check_exception, - ) class SimpleStatefulProcessorWithInitialState(StatefulProcessor): @@ -1642,11 +1604,11 @@ class TTLStatefulProcessor(StatefulProcessor): def init(self, handle: StatefulProcessorHandle) -> None: state_schema = StructType([StructField("value", IntegerType(), True)]) user_key_schema = StructType([StructField("id", StringType(), True)]) - self.ttl_count_state = handle.getValueState("ttl-state", state_schema, 10000) + self.ttl_count_state = handle.getValueState("ttl-state", state_schema, 1000) self.count_state = handle.getValueState("state", state_schema) - self.ttl_list_state = handle.getListState("ttl-list-state", state_schema, 10000) + self.ttl_list_state = handle.getListState("ttl-list-state", state_schema, 1000) self.ttl_map_state = handle.getMapState( - "ttl-map-state", user_key_schema, state_schema, 10000 + "ttl-map-state", user_key_schema, state_schema, 1000 ) def handleInputRows(self, key, rows, timerValues) -> Iterator[pd.DataFrame]: diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index f084318a2967d..198e89cbbcbd3 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1032,8 +1032,8 @@ class SparkConnectPlanner( val outputSchema = parseSchema(rel.getOutputSchema) if (rel.hasInitialInput) { - val initialGroupingCols = rel.getInitialGroupingExpressionsList.asScala.toSeq.map( - expr => Column(transformExpression(expr))) + val initialGroupingCols = rel.getInitialGroupingExpressionsList.asScala.toSeq.map(expr => + Column(transformExpression(expr))) val input = Dataset .ofRows(session, transformRelation(rel.getInput)) @@ -1054,7 +1054,8 @@ class SparkConnectPlanner( outputSchema, rel.getOutputMode, rel.getTimeMode, - initialStateDs, rel.getEventTimeColName) + initialStateDs, + rel.getEventTimeColName) .logicalPlan } else { Dataset @@ -1065,7 +1066,8 @@ class SparkConnectPlanner( outputSchema, rel.getOutputMode, rel.getTimeMode, - null, rel.getEventTimeColName) + null, + rel.getEventTimeColName) .logicalPlan } } From a90e1c5d54e09acacfeaa8be8e55318cb98f10e1 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 29 Jan 2025 15:29:31 -0800 Subject: [PATCH 10/20] add a comment --- .../pandas/test_pandas_transform_with_state.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 03797e0e6712f..e379ac96eb907 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -69,6 +69,7 @@ def conf(cls): ) cfg.set("spark.sql.execution.arrow.transformWithStateInPandas.maxRecordsPerBatch", "2") cfg.set("spark.sql.session.timeZone", "UTC") + # TODO SPARK-50180 this config is to allow tests suite to stop query from FEB sink gracefully cfg.set("spark.sql.streaming.noDataMicroBatches.enabled", "false") return cfg @@ -888,7 +889,9 @@ def check_results(batch_df, batch_id): } else: # check for state metadata source - metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load( + checkpoint_path + ) assert set( metadata_df.select( "operatorId", @@ -1017,7 +1020,9 @@ def check_results(batch_df, batch_id): } else: # check for state metadata source - metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load( + checkpoint_path + ) operator_properties_json_obj = json.loads( metadata_df.select("operatorProperties").collect()[0][0] ) @@ -1099,7 +1104,9 @@ def check_results(batch_df, batch_id): } # check for state metadata source - metadata_df = batch_df.sparkSession.read.format("state-metadata").load(checkpoint_path) + metadata_df = batch_df.sparkSession.read.format("state-metadata").load( + checkpoint_path + ) operator_properties_json_obj = json.loads( metadata_df.select("operatorProperties").collect()[0][0] ) From 7c1cf9d6d4ce51c99c649e8c01e372fe7b4fb75f Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 29 Jan 2025 15:30:44 -0800 Subject: [PATCH 11/20] wording --- .../sql/tests/pandas/test_pandas_transform_with_state.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index e379ac96eb907..cdd122089e1dc 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -69,7 +69,7 @@ def conf(cls): ) cfg.set("spark.sql.execution.arrow.transformWithStateInPandas.maxRecordsPerBatch", "2") cfg.set("spark.sql.session.timeZone", "UTC") - # TODO SPARK-50180 this config is to allow tests suite to stop query from FEB sink gracefully + # TODO SPARK-50180 this config is to stop query from FEB sink gracefully cfg.set("spark.sql.streaming.noDataMicroBatches.enabled", "false") return cfg From b6751bf503f73297ec45f31cb42e704cd70492ca Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 29 Jan 2025 16:00:46 -0800 Subject: [PATCH 12/20] fix metadata suite --- .../test_pandas_transform_with_state.py | 30 ++++++++----------- 1 file changed, 13 insertions(+), 17 deletions(-) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index cdd122089e1dc..74e3d8dbd29a7 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -591,7 +591,7 @@ def _test_transform_with_state_init_state_in_pandas( df = self._build_test_df(input_path) self.assertTrue(df.isStreaming) - output_schema = "id string, value str" + output_schema = "id string, value string" if initial_state is None: data = [("0", 789), ("3", 987)] @@ -879,6 +879,18 @@ def test_transform_with_state_in_pandas_batch_query_initial_state(self): "COVERAGE_PROCESS_START" in os.environ, "Flaky with coverage enabled, skipping for now." ) def test_transform_with_map_state_metadata(self): + self._test_transform_with_map_state_metadata(None) + + def test_transform_with_map_state_metadata_with_init_state(self): + # run the same test suite again but with no-op initial state + # TWS with initial state is using a different python runner + init_data = [("0", 789), ("3", 987)] + initial_state = self.spark.createDataFrame(init_data, "id string, temperature int").groupBy( + "id" + ) + self._test_transform_with_map_state_metadata(initial_state) + + def _test_transform_with_map_state_metadata(self, initial_state): checkpoint_path = tempfile.mktemp() def check_results(batch_df, batch_id): @@ -983,22 +995,6 @@ def check_results(batch_df, batch_id): ) assert list_state_df.isEmpty() - self._test_transform_with_state_in_pandas_basic( - MapStateLargeTTLProcessor(), - check_results, - True, - "processingTime", - checkpoint_path=checkpoint_path, - initial_state=None, - ) - - # run the same test suite again but with no-op initial state - # TWS with initial state is using a different python runner - init_data = [("0", 789), ("3", 987)] - initial_state = self.spark.createDataFrame(init_data, "id string, temperature int").groupBy( - "id" - ) - self._test_transform_with_state_in_pandas_basic( MapStateLargeTTLProcessor(), check_results, From 314bbd2f02751c9b9adc0f21fb33cca07dbdd858 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 30 Jan 2025 11:10:32 -0800 Subject: [PATCH 13/20] fix linter --- python/pyspark/sql/streaming/stateful_processor_util.py | 3 +-- .../sql/tests/pandas/test_pandas_transform_with_state.py | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index df547145eef8a..54cde4113f265 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -17,7 +17,7 @@ from enum import Enum import itertools -from typing import Any, Iterator, List, Optional, Union, TYPE_CHECKING, cast +from typing import Optional, TYPE_CHECKING from pyspark.sql.streaming.stateful_processor_api_client import ( StatefulProcessorApiClient, StatefulProcessorHandleState, @@ -28,7 +28,6 @@ StatefulProcessorHandle, TimerValues, ) -from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle if TYPE_CHECKING: from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 74e3d8dbd29a7..b696ef970b778 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -1376,7 +1376,7 @@ def check_basic_state(batch_df, batch_id): if not isinstance(e, StreamingQueryException): return False - error_msg = str(error) + error_msg = str(e) assert ( "[STREAM_FAILED]" in error_msg and "[STATE_STORE_INVALID_VALUE_SCHEMA_EVOLUTION]" in error_msg From 960de22eb661b7de6d7e82d6bb2de4b6088c2155 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 30 Jan 2025 12:56:03 -0800 Subject: [PATCH 14/20] add back import --- python/pyspark/sql/streaming/stateful_processor_util.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index 54cde4113f265..f7fb7f16deb69 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -17,7 +17,7 @@ from enum import Enum import itertools -from typing import Optional, TYPE_CHECKING +from typing import Any, Iterator, Optional, TYPE_CHECKING from pyspark.sql.streaming.stateful_processor_api_client import ( StatefulProcessorApiClient, StatefulProcessorHandleState, From efe3e15d644fe3c14df1fc26c7dd4cfa9db787d8 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 30 Jan 2025 14:26:03 -0800 Subject: [PATCH 15/20] typo in doc --- python/pyspark/sql/connect/group.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index d7b1f46250126..0301cf0bd1f2b 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -417,7 +417,7 @@ def transformWithStateInPandas( session=self._df._session, ) - applyInPandasWithState.__doc__ = PySparkGroupedData.applyInPandasWithState.__doc__ + transformWithStateInPandas.__doc__ = PySparkGroupedData.transformWithStateInPandas.__doc__ def applyInArrow( self, func: "ArrowGroupedMapFunction", schema: Union[StructType, str] From a13c4e4f3c200facba42e219ac0f1dcb48e1b505 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 30 Jan 2025 14:28:36 -0800 Subject: [PATCH 16/20] linter --- python/pyspark/sql/connect/group.py | 2 +- python/pyspark/sql/connect/plan.py | 1 - python/pyspark/sql/pandas/group_ops.py | 4 ++-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 0301cf0bd1f2b..b73834b145f1d 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -41,7 +41,7 @@ from pyspark.sql.column import Column from pyspark.sql.connect.functions import builtin as F from pyspark.errors import PySparkNotImplementedError, PySparkTypeError -from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle +from pyspark.sql.streaming.stateful_processor import StatefulProcessor if TYPE_CHECKING: from pyspark.sql.connect._typing import ( diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 1fec45088e477..59ca32f63f2ed 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -59,7 +59,6 @@ PySparkValueError, PySparkPicklingError, ) -from pyspark.sql.streaming.stateful_processor import StatefulProcessor if TYPE_CHECKING: from pyspark.sql.connect.client import SparkConnectClient diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 5a03a7b98ff68..1fc63ce7b4171 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -15,7 +15,7 @@ # limitations under the License. # import sys -from typing import Any, Iterator, List, Optional, Union, TYPE_CHECKING, cast +from typing import List, Optional, Union, TYPE_CHECKING, cast import warnings from pyspark.errors import PySparkTypeError @@ -23,7 +23,7 @@ from pyspark.sql.column import Column from pyspark.sql.dataframe import DataFrame from pyspark.sql.streaming.state import GroupStateTimeout -from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle +from pyspark.sql.streaming.stateful_processor import StatefulProcessor from pyspark.sql.types import StructType if TYPE_CHECKING: From 9b249a5367e9d41b7f9bb20b0265ba87614bd099 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 3 Feb 2025 11:28:56 -0800 Subject: [PATCH 17/20] fix linter, address comments --- python/pyspark/sql/connect/group.py | 4 +- python/pyspark/sql/connect/plan.py | 2 +- python/pyspark/sql/pandas/group_ops.py | 2 +- .../sql/streaming/stateful_processor_util.py | 11 +++- ...test_parity_pandas_transform_with_state.py | 17 +++++- .../test_pandas_transform_with_state.py | 56 +++++++++++++------ .../connect/planner/SparkConnectPlanner.scala | 10 ++-- 7 files changed, 73 insertions(+), 29 deletions(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index b73834b145f1d..d7721f1cb7927 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -378,7 +378,7 @@ def transformWithStateInPandas( udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) if initialState is None: udf_obj = UserDefinedFunction( - udf_util.transformWithStateUDF, # type: ignore + udf_util.transformWithStateUDF, returnType=outputStructType, evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, ) @@ -388,7 +388,7 @@ def transformWithStateInPandas( else: self._df._check_same_session(initialState._df) udf_obj = UserDefinedFunction( - udf_util.transformWithStateWithInitStateUDF, # type: ignore + udf_util.transformWithStateWithInitStateUDF, returnType=outputStructType, evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF, ) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 59ca32f63f2ed..3e0872d05a5ea 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2590,10 +2590,10 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: ) # fill in initial state related fields if self._initial_state_plan is not None: - self._initial_state_plan = cast(LogicalPlan, self._initial_state_plan) plan.transform_with_state_in_pandas.initial_input.CopyFrom( self._initial_state_plan.plan(session) ) + assert self._initial_state_grouping_cols is not None plan.transform_with_state_in_pandas.initial_grouping_expressions.extend( [c.to_plan(session) for c in self._initial_state_grouping_cols] ) diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index 1fc63ce7b4171..cd384000f8593 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -363,7 +363,7 @@ def transformWithStateInPandas( timeMode: str, initialState: Optional["GroupedData"] = None, eventTimeColumnName: str = "", - ) -> DataFrame: + ) -> "DataFrame": """ Invokes methods defined in the stateful processor used in arbitrary state API v2. It requires protobuf, pandas and pyarrow as dependencies to process input/state data. We diff --git a/python/pyspark/sql/streaming/stateful_processor_util.py b/python/pyspark/sql/streaming/stateful_processor_util.py index f7fb7f16deb69..fbc3093f87092 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -31,13 +31,17 @@ if TYPE_CHECKING: from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike - from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode # This file places the utilities for transformWithStateInPandas; we have a separate file to avoid # putting internal classes to the stateful_processor.py file which contains public APIs. class TransformWithStateInPandasFuncMode(Enum): + """ + Internal mode for python worker UDF mode for transformWithStateInPandas; external mode are in + `StatefulProcessorHandleState` for public use purposes. + """ + PROCESS_DATA = 1 PROCESS_TIMER = 2 COMPLETE = 3 @@ -45,6 +49,11 @@ class TransformWithStateInPandasFuncMode(Enum): class TransformWithStateInPandasUdfUtils: + """ + Internal Utility class used for python worker UDF for transformWithStateInPandas. This class is + shared for both classic and spark connect mode. + """ + def __init__(self, stateful_processor: StatefulProcessor, time_mode: str): self._stateful_processor = stateful_processor self._time_mode = time_mode diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py index ea7041675b955..e3aeecf930532 100644 --- a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -19,13 +19,28 @@ from pyspark.sql.tests.pandas.test_pandas_transform_with_state import ( TransformWithStateInPandasTestsMixin, ) +from pyspark import SparkConf from pyspark.testing.connectutils import ReusedConnectTestCase class TransformWithStateInPandasParityTests( TransformWithStateInPandasTestsMixin, ReusedConnectTestCase ): - pass + @classmethod + def conf(cls): + cfg = SparkConf(loadDefaults=False) + # explicitly setting configs in both TransformWithStateInPandasTestsMixin and ReusedConnectTestCase + for base in cls.__bases__: + if hasattr(base, "conf"): + parent_cfg = base.conf() + for k, v in parent_cfg.getAll(): + cfg.set(k, v) + + # Extra removing config for connect suites + if cfg._jconf is not None: + cfg._jconf.remove("spark.master") + + return cfg if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index b696ef970b778..d0dbd24113084 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -59,9 +59,7 @@ class TransformWithStateInPandasTestsMixin: @classmethod def conf(cls): - cfg = SparkConf( - loadDefaults=False - ) # Avoid loading default configs so that connect suites can run + cfg = SparkConf() cfg.set("spark.sql.shuffle.partitions", "5") cfg.set( "spark.sql.streaming.stateStore.providerClass", @@ -323,6 +321,8 @@ def check_results(batch_df, batch_id): SimpleTTLStatefulProcessor(), check_results, False, "processingTime" ) + # TODO SPARK-50908 holistic fix for TTL suite + @unittest.skip("test is flaky and it is only a timing issue, skipping until we can resolve") def test_value_state_ttl_expiration(self): def check_results(batch_df, batch_id): if batch_id == 0: @@ -339,24 +339,44 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=1), ], ) + elif batch_id == 1: + assertDataFrameEqual( + batch_df, + [ + Row(id="ttl-count-0", count=2), + Row(id="count-0", count=2), + Row(id="ttl-list-state-count-0", count=3), + Row(id="ttl-map-state-count-0", count=2), + Row(id="ttl-count-1", count=2), + Row(id="count-1", count=2), + Row(id="ttl-list-state-count-1", count=3), + Row(id="ttl-map-state-count-1", count=2), + ], + ) else: + # ttl-count-0 expire and restart from count 0. + # The TTL for value state ttl_count_state gets reset in batch 1 because of the + # update operation and ttl-count-1 keeps the state. + # ttl-list-state-count-0 expire and restart from count 0. + # The TTL for list state ttl_list_state gets reset in batch 1 because of the + # put operation and ttl-list-state-count-1 keeps the state. + # non-ttl state never expires assertDataFrameEqual( batch_df, [ Row(id="ttl-count-0", count=1), - Row(id="count-0", count=2), + Row(id="count-0", count=3), Row(id="ttl-list-state-count-0", count=1), Row(id="ttl-map-state-count-0", count=1), - Row(id="ttl-count-1", count=1), - Row(id="count-1", count=2), - Row(id="ttl-list-state-count-1", count=1), - Row(id="ttl-map-state-count-1", count=1), + Row(id="ttl-count-1", count=3), + Row(id="count-1", count=3), + Row(id="ttl-list-state-count-1", count=7), + Row(id="ttl-map-state-count-1", count=3), ], ) - if batch_id == 0: - # let ttl state expires - time.sleep(2) + if batch_id == 0 or batch_id == 1: + time.sleep(4) input_dir = tempfile.TemporaryDirectory() input_path = input_dir.name @@ -364,6 +384,7 @@ def check_results(batch_df, batch_id): df = self._build_test_df(input_path) self._prepare_input_data(input_path + "/batch1.txt", [1, 0], [0, 0]) self._prepare_input_data(input_path + "/batch2.txt", [1, 0], [0, 0]) + self._prepare_input_data(input_path + "/batch3.txt", [1, 0], [0, 0]) for q in self.spark.streams.active: q.stop() output_schema = StructType( @@ -1278,6 +1299,7 @@ def test_schema_evolution_scenarios(self): input_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) + df = self._build_test_df(input_path) def check_basic_state(batch_df, batch_id): @@ -1286,6 +1308,7 @@ def check_basic_state(batch_df, batch_id): assert result.value["name"] == "name-0" self._run_evolution_test(BasicProcessor(), checkpoint_dir, check_basic_state, df) + self._prepare_test_resource2(input_path) # Test 2: Add fields @@ -1304,9 +1327,7 @@ def check_add_fields(batch_df, batch_id): def check_remove_fields(batch_df, batch_id): result = batch_df.collect()[0] assert result.value["id"] == 0 # First ID from test data - assert ( - result.value["name"] == "name-00" - ), f"batch id: {batch_id}, real df: {batch_df.collect()}" + assert result.value["name"] == "name-00" self._run_evolution_test( RemoveFieldsProcessor(), checkpoint_dir, check_remove_fields, df @@ -1339,6 +1360,7 @@ def test_schema_evolution_fails(self): with tempfile.TemporaryDirectory() as checkpoint_dir: input_path = tempfile.mkdtemp() self._prepare_test_resource1(input_path) + df = self._build_test_df(input_path) def check_add_fields(batch_df, batch_id): @@ -1607,11 +1629,11 @@ class TTLStatefulProcessor(StatefulProcessor): def init(self, handle: StatefulProcessorHandle) -> None: state_schema = StructType([StructField("value", IntegerType(), True)]) user_key_schema = StructType([StructField("id", StringType(), True)]) - self.ttl_count_state = handle.getValueState("ttl-state", state_schema, 1000) + self.ttl_count_state = handle.getValueState("ttl-state", state_schema, 10000) self.count_state = handle.getValueState("state", state_schema) - self.ttl_list_state = handle.getListState("ttl-list-state", state_schema, 1000) + self.ttl_list_state = handle.getListState("ttl-list-state", state_schema, 10000) self.ttl_map_state = handle.getMapState( - "ttl-map-state", user_key_schema, state_schema, 1000 + "ttl-map-state", user_key_schema, state_schema, 10000 ) def handleInputRows(self, key, rows, timerValues) -> Iterator[pd.DataFrame]: diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 198e89cbbcbd3..79202242fe627 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1028,6 +1028,9 @@ class SparkConnectPlanner( val pythonUdf = transformPythonUDF(rel.getTransformWithStateUdf) val cols = rel.getGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) + val input = Dataset + .ofRows(session, transformRelation(rel.getInput)) + .groupBy(cols: _*) val outputSchema = parseSchema(rel.getOutputSchema) @@ -1035,9 +1038,6 @@ class SparkConnectPlanner( val initialGroupingCols = rel.getInitialGroupingExpressionsList.asScala.toSeq.map(expr => Column(transformExpression(expr))) - val input = Dataset - .ofRows(session, transformRelation(rel.getInput)) - .groupBy(cols: _*) val initialStateDs = Dataset .ofRows(session, transformRelation(rel.getInitialInput)) .groupBy(initialGroupingCols: _*) @@ -1058,9 +1058,7 @@ class SparkConnectPlanner( rel.getEventTimeColName) .logicalPlan } else { - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .groupBy(cols: _*) + input .transformWithStateInPandas( Column(pythonUdf), outputSchema, From d17e7d0fc265eaf2b4077a95b2e83796d2d244a3 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 3 Feb 2025 13:10:59 -0800 Subject: [PATCH 18/20] remove tws from missing check compatibility test --- .../connect/pandas/test_parity_pandas_transform_with_state.py | 3 ++- python/pyspark/sql/tests/test_connect_compatibility.py | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py index e3aeecf930532..7e54855853926 100644 --- a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -29,7 +29,8 @@ class TransformWithStateInPandasParityTests( @classmethod def conf(cls): cfg = SparkConf(loadDefaults=False) - # explicitly setting configs in both TransformWithStateInPandasTestsMixin and ReusedConnectTestCase + # explicitly setting configs in both TransformWithStateInPandasTestsMixin and + # ReusedConnectTestCase for base in cls.__bases__: if hasattr(base, "conf"): parent_cfg = base.conf() diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index 37105ee04038e..b2e0cc6229c4b 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -395,7 +395,7 @@ def test_grouping_compatibility(self): """Test Grouping compatibility between classic and connect.""" expected_missing_connect_properties = set() expected_missing_classic_properties = set() - expected_missing_connect_methods = {"transformWithStateInPandas"} + expected_missing_connect_methods = set() expected_missing_classic_methods = set() self.check_compatibility( ClassicGroupedData, From 73ef17751fccd91cb3536dd8e7529eb2ed62ef89 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 5 Feb 2025 12:17:37 -0800 Subject: [PATCH 19/20] address comments --- python/pyspark/sql/connect/group.py | 1 - .../connect/pandas/test_parity_pandas_transform_with_state.py | 4 ++++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index d7721f1cb7927..e9298738d0cf7 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -384,7 +384,6 @@ def transformWithStateInPandas( ) initial_state_plan = None initial_state_grouping_cols = None - else: self._df._check_same_session(initialState._df) udf_obj = UserDefinedFunction( diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py index 7e54855853926..587ae80a8c5e5 100644 --- a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -26,6 +26,10 @@ class TransformWithStateInPandasParityTests( TransformWithStateInPandasTestsMixin, ReusedConnectTestCase ): + """ + Spark connect parity tests for TransformWithStateInPandas. Run every test case in + `TransformWithStateInPandasTestsMixin` in spark connect mode. + """ @classmethod def conf(cls): cfg = SparkConf(loadDefaults=False) From 714f300fcf959315b435bbf080b31c020867f547 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 5 Feb 2025 12:18:08 -0800 Subject: [PATCH 20/20] lint --- .../connect/pandas/test_parity_pandas_transform_with_state.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py index 587ae80a8c5e5..a32bdc1b9b933 100644 --- a/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -30,6 +30,7 @@ class TransformWithStateInPandasParityTests( Spark connect parity tests for TransformWithStateInPandas. Run every test case in `TransformWithStateInPandasTestsMixin` in spark connect mode. """ + @classmethod def conf(cls): cfg = SparkConf(loadDefaults=False)