diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index f1088b212f807..3b3d2d94e2ddf 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1095,6 +1095,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/connect/group.py b/python/pyspark/sql/connect/group.py index 11adc8850fec1..79cf930fd8ee1 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -41,6 +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 if TYPE_CHECKING: from pyspark.sql.connect._typing import ( @@ -361,6 +362,56 @@ 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 + from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasUdfUtils + + udf_util = TransformWithStateInPandasUdfUtils(statefulProcessor, timeMode) + if initialState is None: + udf_obj = UserDefinedFunction( + udf_util.transformWithStateUDF, + returnType=outputStructType, + evalType=PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF, + ) + initial_state_plan = None + initial_state_grouping_cols = None + else: + self._df._check_same_session(initialState._df) + udf_obj = UserDefinedFunction( + udf_util.transformWithStateWithInitStateUDF, + 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 + + return DataFrame( + plan.TransformWithStateInPandas( + child=self._df._plan, + grouping_cols=self._grouping_cols, + function=udf_obj, + output_schema=outputStructType, + 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, + ), + session=self._df._session, + ) + + transformWithStateInPandas.__doc__ = PySparkGroupedData.transformWithStateInPandas.__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..c4c7a6a636307 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -2546,6 +2546,74 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return self._with_relations(plan, session) +class TransformWithStateInPandas(LogicalPlan): + """Logical plan object for a TransformWithStateInPandas.""" + + def __init__( + self, + child: Optional["LogicalPlan"], + grouping_cols: Sequence[Column], + function: "UserDefinedFunction", + output_schema: Union[DataType, 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) + 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: DataType = ( + UnparsedDataType(output_schema) if isinstance(output_schema, str) else 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 + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.group_map.input.CopyFrom(self._child.plan(session)) + plan.group_map.grouping_expressions.extend( + [c.to_plan(session) for c in self._grouping_cols] + ) + plan.group_map.output_mode = self._output_mode + + # fill in initial state related fields + if self._initial_state_plan is not None: + plan.group_map.initial_input.CopyFrom(self._initial_state_plan.plan(session)) + assert self._initial_state_grouping_cols is not None + plan.group_map.initial_grouping_expressions.extend( + [c.to_plan(session) for c in self._initial_state_grouping_cols] + ) + + # fill in transformWithStateInPandas related fields + tws_info = proto.TransformWithStateInfo() + tws_info.time_mode = self._time_mode + tws_info.event_time_column_name = self._event_time_col_name + tws_info.output_schema.CopyFrom(pyspark_types_to_proto_types(self._output_schema)) + + plan.group_map.transform_with_state_info.CopyFrom(tws_info) + + # wrap transformWithStateInPandasUdf in a function + plan.group_map.func.CopyFrom(self._function.to_plan_udf(session)) + + 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..525ba88ff67c6 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"\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"\xd2\x06\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x12\x65\n\x19transform_with_state_info\x18\x0b \x01(\x0b\x32%.spark.connect.TransformWithStateInfoH\x04R\x16transformWithStateInfo\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schemaB\x1c\n\x1a_transform_with_state_info"\xdf\x01\n\x16TransformWithStateInfo\x12\x1b\n\ttime_mode\x18\x01 \x01(\tR\x08timeMode\x12\x38\n\x16\x65vent_time_column_name\x18\x02 \x01(\tH\x00R\x13\x65ventTimeColumnName\x88\x01\x01\x12\x41\n\routput_schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x01R\x0coutputSchema\x88\x01\x01\x42\x19\n\x17_event_time_column_nameB\x10\n\x0e_output_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -219,29 +219,31 @@ _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["_GROUPMAP"]._serialized_end = 14749 + _globals["_TRANSFORMWITHSTATEINFO"]._serialized_start = 14752 + _globals["_TRANSFORMWITHSTATEINFO"]._serialized_end = 14975 + _globals["_COGROUPMAP"]._serialized_start = 14978 + _globals["_COGROUPMAP"]._serialized_end = 15504 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15507 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15864 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15867 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 16111 + _globals["_PYTHONUDTF"]._serialized_start = 16114 + _globals["_PYTHONUDTF"]._serialized_end = 16291 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 16294 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16445 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16447 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16522 + _globals["_COLLECTMETRICS"]._serialized_start = 16525 + _globals["_COLLECTMETRICS"]._serialized_end = 16661 + _globals["_PARSE"]._serialized_start = 16664 + _globals["_PARSE"]._serialized_end = 17052 _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["_PARSE_PARSEFORMAT"]._serialized_start = 16953 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 17041 + _globals["_ASOFJOIN"]._serialized_start = 17055 + _globals["_ASOFJOIN"]._serialized_end = 17530 + _globals["_LATERALJOIN"]._serialized_start = 17533 + _globals["_LATERALJOIN"]._serialized_end = 17763 # @@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..beeeb712da762 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -3613,6 +3613,7 @@ class GroupMap(google.protobuf.message.Message): OUTPUT_MODE_FIELD_NUMBER: builtins.int TIMEOUT_CONF_FIELD_NUMBER: builtins.int STATE_SCHEMA_FIELD_NUMBER: builtins.int + TRANSFORM_WITH_STATE_INFO_FIELD_NUMBER: builtins.int @property def input(self) -> global___Relation: """(Required) Input relation for Group Map API: apply, applyInPandas.""" @@ -3654,6 +3655,11 @@ class GroupMap(google.protobuf.message.Message): @property def state_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: """(Optional) The schema for the grouped state.""" + @property + def transform_with_state_info(self) -> global___TransformWithStateInfo: + """Below fields are used by TransformWithState and TransformWithStateInPandas + (Optional) TransformWithState related parameters. + """ def __init__( self, *, @@ -3677,6 +3683,7 @@ class GroupMap(google.protobuf.message.Message): output_mode: builtins.str | None = ..., timeout_conf: builtins.str | None = ..., state_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., + transform_with_state_info: global___TransformWithStateInfo | None = ..., ) -> None: ... def HasField( self, @@ -3689,6 +3696,8 @@ class GroupMap(google.protobuf.message.Message): b"_state_schema", "_timeout_conf", b"_timeout_conf", + "_transform_with_state_info", + b"_transform_with_state_info", "func", b"func", "initial_input", @@ -3703,6 +3712,8 @@ class GroupMap(google.protobuf.message.Message): b"state_schema", "timeout_conf", b"timeout_conf", + "transform_with_state_info", + b"transform_with_state_info", ], ) -> builtins.bool: ... def ClearField( @@ -3716,6 +3727,8 @@ class GroupMap(google.protobuf.message.Message): b"_state_schema", "_timeout_conf", b"_timeout_conf", + "_transform_with_state_info", + b"_transform_with_state_info", "func", b"func", "grouping_expressions", @@ -3736,6 +3749,8 @@ class GroupMap(google.protobuf.message.Message): b"state_schema", "timeout_conf", b"timeout_conf", + "transform_with_state_info", + b"transform_with_state_info", ], ) -> None: ... @typing.overload @@ -3757,9 +3772,82 @@ class GroupMap(google.protobuf.message.Message): def WhichOneof( self, oneof_group: typing_extensions.Literal["_timeout_conf", b"_timeout_conf"] ) -> typing_extensions.Literal["timeout_conf"] | None: ... + @typing.overload + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "_transform_with_state_info", b"_transform_with_state_info" + ], + ) -> typing_extensions.Literal["transform_with_state_info"] | None: ... global___GroupMap = GroupMap +class TransformWithStateInfo(google.protobuf.message.Message): + """Additional input parameters used for TransformWithState operator.""" + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TIME_MODE_FIELD_NUMBER: builtins.int + EVENT_TIME_COLUMN_NAME_FIELD_NUMBER: builtins.int + OUTPUT_SCHEMA_FIELD_NUMBER: builtins.int + time_mode: builtins.str + """(Required) Time mode string for transformWithState.""" + event_time_column_name: builtins.str + """(Optional) Event time column name.""" + @property + def output_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: + """(Optional) Schema for the output DataFrame. + Only required used for TransformWithStateInPandas. + """ + def __init__( + self, + *, + time_mode: builtins.str = ..., + event_time_column_name: builtins.str | None = ..., + output_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_event_time_column_name", + b"_event_time_column_name", + "_output_schema", + b"_output_schema", + "event_time_column_name", + b"event_time_column_name", + "output_schema", + b"output_schema", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_event_time_column_name", + b"_event_time_column_name", + "_output_schema", + b"_output_schema", + "event_time_column_name", + b"event_time_column_name", + "output_schema", + b"output_schema", + "time_mode", + b"time_mode", + ], + ) -> None: ... + @typing.overload + def WhichOneof( + self, + oneof_group: typing_extensions.Literal[ + "_event_time_column_name", b"_event_time_column_name" + ], + ) -> typing_extensions.Literal["event_time_column_name"] | None: ... + @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_output_schema", b"_output_schema"] + ) -> typing_extensions.Literal["output_schema"] | None: ... + +global___TransformWithStateInfo = TransformWithStateInfo + class CoGroupMap(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index e0108da34f0c2..cd384000f8593 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -14,9 +14,8 @@ # 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 +from typing import List, Optional, Union, TYPE_CHECKING, cast import warnings from pyspark.errors import PySparkTypeError @@ -24,18 +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_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.streaming.stateful_processor import StatefulProcessor from pyspark.sql.types import StructType if TYPE_CHECKING: @@ -46,7 +34,6 @@ PandasCogroupedMapFunction, ArrowGroupedMapFunction, ArrowCogroupedMapFunction, - DataFrameLike as PandasDataFrameLike, ) from pyspark.sql.group import GroupedData @@ -376,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 @@ -497,6 +484,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 +492,20 @@ 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 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, - 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 cd2e1230418f3..55a4dd0f8e7d2 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, @@ -1180,6 +1179,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): """ @@ -1235,6 +1235,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..fbc3093f87092 100644 --- a/python/pyspark/sql/streaming/stateful_processor_util.py +++ b/python/pyspark/sql/streaming/stateful_processor_util.py @@ -16,13 +16,220 @@ # from enum import Enum +import itertools +from typing import Any, Iterator, Optional, TYPE_CHECKING +from pyspark.sql.streaming.stateful_processor_api_client import ( + StatefulProcessorApiClient, + StatefulProcessorHandleState, +) +from pyspark.sql.streaming.stateful_processor import ( + ExpiredTimerInfo, + StatefulProcessor, + StatefulProcessorHandle, + TimerValues, +) + +if TYPE_CHECKING: + from pyspark.sql.pandas._typing import DataFrameLike as PandasDataFrameLike # 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 PRE_INIT = 4 + + +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 + + 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) 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..fb25c448cef0a --- /dev/null +++ b/python/pyspark/sql/tests/connect/pandas/test_parity_pandas_transform_with_state.py @@ -0,0 +1,61 @@ +# +# 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 import SparkConf +from pyspark.testing.connectutils import ReusedConnectTestCase + + +class TransformWithStateInPandasParityTests( + TransformWithStateInPandasTestsMixin, ReusedConnectTestCase +): + """ + Spark connect parity tests for TransformWithStateInPandas. Run every test case in + `TransformWithStateInPandasTestsMixin` in spark connect mode. + """ + + @classmethod + def conf(cls): + # Due to multiple inheritance from the same level, we need to explicitly setting configs in + # both TransformWithStateInPandasTestsMixin and ReusedConnectTestCase here + cfg = SparkConf(loadDefaults=False) + 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__": + 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/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index a9a0bbb31d492..3257430d45e94 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,8 @@ def conf(cls): ) cfg.set("spark.sql.execution.arrow.transformWithStateInPandas.maxRecordsPerBatch", "2") cfg.set("spark.sql.session.timeZone", "UTC") + # TODO SPARK-49046 this config is to stop query from FEB sink gracefully + cfg.set("spark.sql.streaming.noDataMicroBatches.enabled", "false") return cfg def _prepare_input_data(self, input_path, col1, col2): @@ -270,14 +272,10 @@ 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: - for q in self.spark.streams.active: - q.stop() + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } self._test_transform_with_state_in_pandas_basic( ListStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -295,14 +293,10 @@ 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: - for q in self.spark.streams.active: - q.stop() + assert set(batch_df.sort("id").collect()) == { + Row(id="0", countAsString="2"), + Row(id="1", countAsString="2"), + } self._test_transform_with_state_in_pandas_basic( MapStateLargeTTLProcessor(), check_results, True, "processingTime" @@ -317,14 +311,11 @@ 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: - for q in self.spark.streams.active: - q.stop() self._test_transform_with_state_in_pandas_basic( SimpleTTLStatefulProcessor(), check_results, False, "processingTime" @@ -362,7 +353,7 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=2), ], ) - elif batch_id == 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. @@ -383,9 +374,7 @@ def check_results(batch_df, batch_id): Row(id="ttl-map-state-count-1", count=3), ], ) - else: - for q in self.spark.streams.active: - q.stop() + if batch_id == 0 or batch_id == 1: time.sleep(4) @@ -469,23 +458,23 @@ def _test_transform_with_state_in_pandas_proc_timer(self, stateful_processor, ch self.assertTrue(q.exception() is None) 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"), @@ -499,28 +488,14 @@ def check_results(batch_df, batch_id): 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) - 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"), 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 - - else: - for q in self.spark.streams.active: - q.stop() self._test_transform_with_state_in_pandas_proc_timer( ProcTimeStatefulProcessor(), check_results @@ -603,7 +578,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 @@ -611,9 +586,6 @@ def check_results(batch_df, batch_id): Row(id="a", timestamp="15"), Row(id="a-expired", timestamp="10000"), } - else: - for q in self.spark.streams.active: - q.stop() self._test_transform_with_state_in_pandas_event_time( EventTimeStatefulProcessor(), check_results @@ -640,12 +612,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 string" if initial_state is None: data = [("0", 789), ("3", 987)] @@ -741,14 +708,10 @@ 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: - for q in self.spark.streams.active: - q.stop() + assert set(batch_df.collect()) == { + Row(id1="0", id2="1", value=str(123 + 46)), + Row(id1="1", id2="2", value=str(146 + 346)), + } self._test_transform_with_state_non_contiguous_grouping_cols( SimpleStatefulProcessorWithInitialState(), check_results @@ -756,15 +719,11 @@ def check_results(batch_df, batch_id): 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: - for q in self.spark.streams.active: - q.stop() + # 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")] @@ -875,16 +834,13 @@ 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: - for q in self.spark.streams.active: - q.stop() self._test_transform_with_state_init_state_in_pandas( StatefulProcessorWithInitialStateTimers(), check_results, "processingTime" @@ -944,6 +900,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): @@ -954,7 +922,9 @@ 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 +965,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 +981,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,31 +1009,13 @@ 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() - for q in self.spark.streams.active: - q.stop() - - 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, @@ -1085,7 +1037,9 @@ 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] ) @@ -1100,7 +1054,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) @@ -1123,7 +1077,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) @@ -1140,9 +1094,6 @@ 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._test_transform_with_state_in_pandas_basic( ListStateProcessor(), check_results, @@ -1170,7 +1121,9 @@ 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] ) @@ -1186,7 +1139,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) @@ -1210,7 +1163,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) @@ -1230,9 +1183,6 @@ def check_results(batch_df, batch_id): .collect() ) - for q in self.spark.streams.active: - q.stop() - with self.sql_conf( {"spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled": "true"} ): @@ -1264,7 +1214,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() @@ -1309,9 +1259,7 @@ def test_transform_with_state_with_timers_single_partition(self): 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), @@ -1323,35 +1271,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""" @@ -1431,7 +1369,6 @@ def check_add_fields(batch_df, batch_id): 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): @@ -1440,7 +1377,6 @@ def check_upcast(batch_df, batch_id): # 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): @@ -1448,28 +1384,27 @@ 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 ( + error_msg = str(e) + 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, - ) - def test_not_nullable_fails(self): with self.sql_conf({"spark.sql.streaming.stateStore.encodingFormat": "avro"}): with tempfile.TemporaryDirectory() as checkpoint_dir: @@ -1483,26 +1418,25 @@ 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): + try: + self._run_evolution_test( + BasicProcessorNotNullable(), + 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 ( + error_msg = str(e) + assert ( "[TRANSFORM_WITH_STATE_SCHEMA_MUST_BE_NULLABLE]" in error_msg and "column family state must be nullable" in error_msg ) - self._run_evolution_test( - BasicProcessorNotNullable(), - checkpoint_dir, - check_basic_state, - df, - check_exception=check_exception, - ) - class SimpleStatefulProcessorWithInitialState(StatefulProcessor): # this dict is the same as input initial state dataframe 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, 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..70a52a2111494 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -1031,6 +1031,23 @@ message GroupMap { // (Optional) The schema for the grouped state. optional DataType state_schema = 10; + + // Below fields are used by TransformWithState and TransformWithStateInPandas + // (Optional) TransformWithState related parameters. + optional TransformWithStateInfo transform_with_state_info = 11; +} + +// Additional input parameters used for TransformWithState operator. +message TransformWithStateInfo { + // (Required) Time mode string for transformWithState. + string time_mode = 1; + + // (Optional) Event time column name. + optional string event_time_column_name = 2; + + // (Optional) Schema for the output DataFrame. + // Only required used for TransformWithStateInPandas. + optional DataType output_schema = 3; } message CoGroupMap { 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 f8fd56f38f9b6..dc8883d72ee60 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 @@ -659,6 +659,10 @@ class SparkConnectPlanner( case PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF => group.flatMapGroupsInArrow(Column(pythonUdf)).logicalPlan + case PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_UDF | + PythonEvalType.SQL_TRANSFORM_WITH_STATE_PANDAS_INIT_STATE_UDF => + transformTransformWithStateInPandas(pythonUdf, group, rel) + case _ => throw InvalidPlanInput( s"Function with EvalType: ${pythonUdf.evalType} is not supported") @@ -1034,6 +1038,57 @@ class SparkConnectPlanner( .logicalPlan } + private def transformTransformWithStateInPandas( + pythonUdf: PythonUDF, + groupedDs: RelationalGroupedDataset, + rel: proto.GroupMap): LogicalPlan = { + val twsInfo = rel.getTransformWithStateInfo + val outputSchema: StructType = { + transformDataType(twsInfo.getOutputSchema) match { + case s: StructType => s + case dt => + throw InvalidPlanInput( + "Invalid user-defined output schema type for TransformWithStateInPandas. " + + s"Expect a struct type, but got ${dt.typeName}.") + } + } + + if (rel.hasInitialInput) { + val initialGroupingCols = rel.getInitialGroupingExpressionsList.asScala.toSeq.map(expr => + Column(transformExpression(expr))) + + 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.getFunc) + .builder(groupedDs.df.logicalPlan.output) + .asInstanceOf[PythonUDF] + + groupedDs + .transformWithStateInPandas( + Column(resolvedPythonUDF), + outputSchema, + rel.getOutputMode, + twsInfo.getTimeMode, + initialStateDs, + twsInfo.getEventTimeColumnName) + .logicalPlan + } else { + groupedDs + .transformWithStateInPandas( + Column(pythonUdf), + outputSchema, + rel.getOutputMode, + twsInfo.getTimeMode, + null, + twsInfo.getEventTimeColumnName) + .logicalPlan + } + } + private def transformCommonInlineUserDefinedTableFunction( fun: proto.CommonInlineUserDefinedTableFunction): LogicalPlan = { fun.getFunctionCase match {