diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index f17734584f154..24d2d6157334e 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -40,7 +40,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xbd\x35\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\xc1\x0f\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x12\x61\n\x11specialized_array\x18\x19 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.SpecializedArrayH\x00R\x10specializedArray\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xc0\x02\n\x10SpecializedArray\x12,\n\x05\x62ools\x18\x01 \x01(\x0b\x32\x14.spark.connect.BoolsH\x00R\x05\x62ools\x12)\n\x04ints\x18\x02 \x01(\x0b\x32\x13.spark.connect.IntsH\x00R\x04ints\x12,\n\x05longs\x18\x03 \x01(\x0b\x32\x14.spark.connect.LongsH\x00R\x05longs\x12/\n\x06\x66loats\x18\x04 \x01(\x0b\x32\x15.spark.connect.FloatsH\x00R\x06\x66loats\x12\x32\n\x07\x64oubles\x18\x05 \x01(\x0b\x32\x16.spark.connect.DoublesH\x00R\x07\x64oubles\x12\x32\n\x07strings\x18\x06 \x01(\x0b\x32\x16.spark.connect.StringsH\x00R\x07stringsB\x0c\n\nvalue_typeB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\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\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\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\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe5\x04\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType\x12\x62\n\x11table_arg_options\x18\x03 \x01(\x0b\x32\x31.spark.connect.SubqueryExpression.TableArgOptionsH\x00R\x0ftableArgOptions\x88\x01\x01\x1a\xea\x01\n\x0fTableArgOptions\x12@\n\x0epartition_spec\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12\x37\n\x15with_single_partition\x18\x03 \x01(\x08H\x00R\x13withSinglePartition\x88\x01\x01\x42\x18\n\x16_with_single_partition"z\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x12\x1b\n\x17SUBQUERY_TYPE_TABLE_ARG\x10\x03\x42\x14\n\x12_table_arg_optionsB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xbd\x35\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\xc1\x0f\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x12\x61\n\x11specialized_array\x18\x19 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.SpecializedArrayH\x00R\x10specializedArray\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xc0\x02\n\x10SpecializedArray\x12,\n\x05\x62ools\x18\x01 \x01(\x0b\x32\x14.spark.connect.BoolsH\x00R\x05\x62ools\x12)\n\x04ints\x18\x02 \x01(\x0b\x32\x13.spark.connect.IntsH\x00R\x04ints\x12,\n\x05longs\x18\x03 \x01(\x0b\x32\x14.spark.connect.LongsH\x00R\x05longs\x12/\n\x06\x66loats\x18\x04 \x01(\x0b\x32\x15.spark.connect.FloatsH\x00R\x06\x66loats\x12\x32\n\x07\x64oubles\x18\x05 \x01(\x0b\x32\x16.spark.connect.DoublesH\x00R\x07\x64oubles\x12\x32\n\x07strings\x18\x06 \x01(\x0b\x32\x16.spark.connect.StringsH\x00R\x07stringsB\x0c\n\nvalue_typeB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\x8d\x03\n\x1f\x43ommonInlineUserDefinedFunction\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\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdf\x12\x1f\n\x0bis_distinct\x18\x07 \x01(\x08R\nisDistinctB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\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\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe5\x04\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType\x12\x62\n\x11table_arg_options\x18\x03 \x01(\x0b\x32\x31.spark.connect.SubqueryExpression.TableArgOptionsH\x00R\x0ftableArgOptions\x88\x01\x01\x1a\xea\x01\n\x0fTableArgOptions\x12@\n\x0epartition_spec\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12\x37\n\x15with_single_partition\x18\x03 \x01(\x08H\x00R\x13withSinglePartition\x88\x01\x01\x42\x18\n\x16_with_single_partition"z\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x12\x1b\n\x17SUBQUERY_TYPE_TABLE_ARG\x10\x03\x42\x14\n\x12_table_arg_optionsB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -110,31 +110,31 @@ _globals["_EXPRESSIONCOMMON"]._serialized_start = 6980 _globals["_EXPRESSIONCOMMON"]._serialized_end = 7045 _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 7048 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 7412 - _globals["_PYTHONUDF"]._serialized_start = 7415 - _globals["_PYTHONUDF"]._serialized_end = 7619 - _globals["_SCALARSCALAUDF"]._serialized_start = 7622 - _globals["_SCALARSCALAUDF"]._serialized_end = 7836 - _globals["_JAVAUDF"]._serialized_start = 7839 - _globals["_JAVAUDF"]._serialized_end = 7988 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7990 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 8089 - _globals["_CALLFUNCTION"]._serialized_start = 8091 - _globals["_CALLFUNCTION"]._serialized_end = 8199 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 8201 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 8293 - _globals["_MERGEACTION"]._serialized_start = 8296 - _globals["_MERGEACTION"]._serialized_end = 8808 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8518 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8624 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8627 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8794 - _globals["_LAZYEXPRESSION"]._serialized_start = 8810 - _globals["_LAZYEXPRESSION"]._serialized_end = 8875 - _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8878 - _globals["_SUBQUERYEXPRESSION"]._serialized_end = 9491 - _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_start = 9111 - _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_end = 9345 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 9347 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 9469 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 7445 + _globals["_PYTHONUDF"]._serialized_start = 7448 + _globals["_PYTHONUDF"]._serialized_end = 7652 + _globals["_SCALARSCALAUDF"]._serialized_start = 7655 + _globals["_SCALARSCALAUDF"]._serialized_end = 7869 + _globals["_JAVAUDF"]._serialized_start = 7872 + _globals["_JAVAUDF"]._serialized_end = 8021 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 8023 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 8122 + _globals["_CALLFUNCTION"]._serialized_start = 8124 + _globals["_CALLFUNCTION"]._serialized_end = 8232 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 8234 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 8326 + _globals["_MERGEACTION"]._serialized_start = 8329 + _globals["_MERGEACTION"]._serialized_end = 8841 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8551 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8657 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8660 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8827 + _globals["_LAZYEXPRESSION"]._serialized_start = 8843 + _globals["_LAZYEXPRESSION"]._serialized_end = 8908 + _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8911 + _globals["_SUBQUERYEXPRESSION"]._serialized_end = 9524 + _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_start = 9144 + _globals["_SUBQUERYEXPRESSION_TABLEARGOPTIONS"]._serialized_end = 9378 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 9380 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 9502 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 63cab8fd9d831..714be5227b20e 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1537,6 +1537,7 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): PYTHON_UDF_FIELD_NUMBER: builtins.int SCALAR_SCALA_UDF_FIELD_NUMBER: builtins.int JAVA_UDF_FIELD_NUMBER: builtins.int + IS_DISTINCT_FIELD_NUMBER: builtins.int function_name: builtins.str """(Required) Name of the user-defined function.""" deterministic: builtins.bool @@ -1552,6 +1553,8 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): def scalar_scala_udf(self) -> global___ScalarScalaUDF: ... @property def java_udf(self) -> global___JavaUDF: ... + is_distinct: builtins.bool + """(Required) Indicate if this function should be applied on distinct values.""" def __init__( self, *, @@ -1561,6 +1564,7 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): python_udf: global___PythonUDF | None = ..., scalar_scala_udf: global___ScalarScalaUDF | None = ..., java_udf: global___JavaUDF | None = ..., + is_distinct: builtins.bool = ..., ) -> None: ... def HasField( self, @@ -1586,6 +1590,8 @@ class CommonInlineUserDefinedFunction(google.protobuf.message.Message): b"function", "function_name", b"function_name", + "is_distinct", + b"is_distinct", "java_udf", b"java_udf", "python_udf", diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala index 55613b2b20134..fc477d1bc5ef5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -406,14 +406,20 @@ object DeserializerBuildHelper { NewInstance(cls, arguments, Nil, propagateNull = false, dt, outerPointerGetter)) case AgnosticEncoders.RowEncoder(fields) => + val isExternalRow = !path.dataType.isInstanceOf[StructType] val convertedFields = fields.zipWithIndex.map { case (f, i) => val newTypePath = walkedTypePath.recordField( f.enc.clsTag.runtimeClass.getName, f.name) - exprs.If( - Invoke(path, "isNullAt", BooleanType, exprs.Literal(i) :: Nil), - exprs.Literal.create(null, externalDataTypeFor(f.enc)), - createDeserializer(f.enc, GetStructField(path, i), newTypePath)) + val deserializer = createDeserializer(f.enc, GetStructField(path, i), newTypePath) + if (isExternalRow) { + exprs.If( + Invoke(path, "isNullAt", BooleanType, exprs.Literal(i) :: Nil), + exprs.Literal.create(null, externalDataTypeFor(f.enc)), + deserializer) + } else { + deserializer + } } exprs.If(IsNull(path), exprs.Literal.create(null, externalDataTypeFor(enc)), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index 5f0b42fec0fa8..b92acfb5b0f3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{DeserializerBuildHelper, InternalRow, Java import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.{Deserializer, Serializer} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, InitializeJavaBean, NewInstance} +import org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull import org.apache.spark.sql.catalyst.optimizer.{ReassignLambdaVariableID, SimplifyCasts} import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LeafNode, LocalRelation} import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -200,8 +200,7 @@ case class ExpressionEncoder[T]( UnresolvedAttribute.quoted(part.toString) case GetStructField(GetColumnByOrdinal(0, dt), ordinal, _) => GetColumnByOrdinal(ordinal, dt) - case If(IsNull(GetColumnByOrdinal(0, _)), _, n: NewInstance) => n - case If(IsNull(GetColumnByOrdinal(0, _)), _, i: InitializeJavaBean) => i + case If(IsNull(GetColumnByOrdinal(0, _)), _, e) => e } } else { // For other input objects like primitive, array, map, etc., we deserialize the first column diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala index bdd1d507e0440..bba784800976c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoderSuite.scala @@ -142,6 +142,23 @@ case class OptionNestedGeneric[T](list: Option[T]) case class MapNestedGenericKey[T](list: Map[T, Int]) case class MapNestedGenericValue[T](list: Map[Int, T]) +class Wrapper[T](val value: T) { + override def hashCode(): Int = value.hashCode() + override def equals(obj: Any): Boolean = obj match { + case other: Wrapper[T @unchecked] => value == other.value + case _ => false + } +} + +class WrapperCodec[T] extends Codec[Wrapper[T], T] { + override def encode(in: Wrapper[T]): T = in.value + override def decode(out: T): Wrapper[T] = new Wrapper(out) +} + +class WrapperCodecProvider[T] extends (() => Codec[Wrapper[T], T]) { + override def apply(): Codec[Wrapper[T], T] = new WrapperCodec[T] +} + class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTest with QueryErrorsBase { OuterScopes.addOuterScope(this) @@ -568,6 +585,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(FooEnum.E1, "scala Enum") + private def testTransformingEncoder( name: String, provider: () => Codec[Any, Array[Byte]]): Unit = test(name) { @@ -585,6 +603,19 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes testTransformingEncoder("transforming java serialization encoder", JavaSerializationCodec) testTransformingEncoder("transforming kryo encoder", KryoSerializationCodec) + test("transforming row encoder") { + val schema = new StructType().add("a", LongType).add("b", StringType) + val encoder = ExpressionEncoder(TransformingEncoder( + classTag[Wrapper[Row]], + RowEncoder.encoderFor(schema), + new WrapperCodecProvider[Row])) + .resolveAndBind() + val toRow = encoder.createSerializer() + val fromRow = encoder.createDeserializer() + assert(fromRow(toRow(new Wrapper(Row(9L, "x")))) == new Wrapper(Row(9L, "x"))) + } + + // Scala / Java big decimals ---------------------------------------------------------- encodeDecodeTest(BigDecimal(("9" * 20) + "." + "9" * 18), diff --git a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/UserDefinedFunctionE2ETestSuite.scala b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/UserDefinedFunctionE2ETestSuite.scala index 71734b907b9f3..81545afadd3f8 100644 --- a/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/UserDefinedFunctionE2ETestSuite.scala +++ b/sql/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/UserDefinedFunctionE2ETestSuite.scala @@ -27,9 +27,9 @@ import org.apache.spark.sql.{AnalysisException, Encoder, Encoders, Row} import org.apache.spark.sql.api.java.UDF2 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{PrimitiveIntEncoder, PrimitiveLongEncoder, StringEncoder} import org.apache.spark.sql.connect.test.{QueryTest, RemoteSparkSession} -import org.apache.spark.sql.expressions.Aggregator -import org.apache.spark.sql.functions.{call_function, col, struct, udaf, udf} -import org.apache.spark.sql.types.{IntegerType, StringType} +import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction} +import org.apache.spark.sql.functions.{call_function, col, count, lit, struct, udaf, udf} +import org.apache.spark.sql.types.{DataType, IntegerType, LongType, StringType, StructField, StructType} /** * All tests in this class requires client UDF defined in this test class synced with the server. @@ -442,6 +442,41 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession .as(StringEncoder) checkDataset(ds, "01", "12") } + + test("inline UserDefinedAggregateFunction") { + val summer0 = new LongSummer(0) + val summer1 = new LongSummer(1) + val summer3 = new LongSummer(3) + val ds = spark + .range(10) + .select( + count(lit(1)), + summer0(), + summer1(col("id")), + summer3(col("id"), col("id") + 1, col("id") + 2)) + checkDataset(ds, Row(10L, 10L, Row(45L, 10L), Row(45L, 55L, 65L, 10L))) + } + + test("inline UserDefinedAggregateFunction distinct") { + val summer1 = new LongSummer(1) + val ds = + spark.range(10).union(spark.range(10)).select(count(lit(1)), summer1.distinct(col("id"))) + checkDataset(ds, Row(20L, Row(45L, 10L))) + } + + test("register UserDefinedAggregateFunction") { + spark.udf.register("s0", new LongSummer(0)) + spark.udf.register("s2", new LongSummer(2)) + spark.udf.register("s4", new LongSummer(4)) + val ds = spark + .range(10) + .select( + count(lit(1)), + call_function("s0"), + call_function("s2", col("id"), col("id") + 1), + call_function("s4", col("id"), col("id") + 1, col("id") + 2, col("id") + 3)) + checkDataset(ds, Row(10L, 10L, Row(45L, 55L, 10L), Row(45L, 55L, 65L, 75L, 10L))) + } } case class UdafTestInput(id: Long, extra: Long) @@ -503,3 +538,56 @@ object RowAggregator extends Aggregator[Row, (Long, Long), Long] { class StringConcat extends UDF2[String, String, String] { override def call(t1: String, t2: String): String = t1 + t2 } + +class LongSummer(size: Int) extends UserDefinedAggregateFunction { + assert(size >= 0) + + override def inputSchema: StructType = { + StructType(Array.tabulate(size)(i => StructField(s"val_$i", LongType))) + } + + override def bufferSchema: StructType = inputSchema.add("counter", LongType) + + override def dataType: DataType = { + if (size == 0) { + LongType + } else { + bufferSchema + } + } + + override def deterministic: Boolean = true + + override def initialize(buffer: MutableAggregationBuffer): Unit = { + var i = 0 + while (i < size + 1) { + buffer.update(i, 0L) + i += 1 + } + } + + override def update(buffer: MutableAggregationBuffer, input: Row): Unit = { + var i = 0 + while (i < size) { + buffer.update(i, buffer.getLong(i) + input.getLong(i)) + i += 1 + } + buffer.update(size, buffer.getLong(size) + 1) + } + + override def merge(buffer1: MutableAggregationBuffer, buffer2: Row): Unit = { + var i = 0 + while (i < size + 1) { + buffer1.update(i, buffer1.getLong(i) + buffer2.getLong(i)) + i += 1 + } + } + + override def evaluate(buffer: Row): Any = { + if (size == 0) { + buffer.getLong(0) + } else { + buffer + } + } +} diff --git a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto index c17c7d3971fc0..77fb3d423a817 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -384,6 +384,8 @@ message CommonInlineUserDefinedFunction { ScalarScalaUDF scalar_scala_udf = 5; JavaUDF java_udf = 6; } + // (Required) Indicate if this function should be applied on distinct values. + bool is_distinct = 7; } message PythonUDF { diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UDFRegistration.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UDFRegistration.scala index 68a216a708024..338909bf1a6c9 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UDFRegistration.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UDFRegistration.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.connect import org.apache.spark.connect.proto import org.apache.spark.sql +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.connect.common.DataTypeProtoConverter -import org.apache.spark.sql.expressions.{UserDefinedAggregateFunction, UserDefinedFunction} +import org.apache.spark.sql.expressions.{UserDefinedAggregateFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.types.DataType /** @@ -55,6 +56,13 @@ class UDFRegistration(session: SparkSession) extends sql.UDFRegistration { /** @inheritdoc */ override def register( name: String, - udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = - throw ConnectClientUnsupportedErrors.registerUdaf() + udaf: UserDefinedAggregateFunction): UserDefinedAggregateFunction = { + val wrapped = UserDefinedAggregator( + aggregator = new UserDefinedAggregateFunctionWrapper(udaf), + inputEncoder = RowEncoder.encoderFor(udaf.inputSchema), + givenName = Option(name), + deterministic = udaf.deterministic) + register(name, wrapped, "scala_udf", validateParameterCount = false) + udaf + } } diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UdfToProtoUtils.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UdfToProtoUtils.scala index 487b165c7fc23..486302266539b 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UdfToProtoUtils.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UdfToProtoUtils.scala @@ -69,10 +69,12 @@ private[sql] object UdfToProtoUtils { */ private[sql] def toProto( udf: UserDefinedFunction, - arguments: Seq[proto.Expression] = Nil): proto.CommonInlineUserDefinedFunction = { + arguments: Seq[proto.Expression] = Nil, + isDistinct: Boolean = false): proto.CommonInlineUserDefinedFunction = { val invokeUdf = proto.CommonInlineUserDefinedFunction .newBuilder() .setDeterministic(udf.deterministic) + .setIsDistinct(isDistinct) .addAllArguments(arguments.asJava) val protoUdf = invokeUdf.getScalarScalaUdfBuilder .setNullable(udf.nullable) diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UserDefinedAggregateFunctionWrapper.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UserDefinedAggregateFunctionWrapper.scala new file mode 100644 index 0000000000000..bcbd45f3a9f1c --- /dev/null +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/UserDefinedAggregateFunctionWrapper.scala @@ -0,0 +1,94 @@ +/* + * 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. + */ +package org.apache.spark.sql.connect + +import scala.reflect.classTag + +import org.apache.spark.sql.{Encoder, Row} +import org.apache.spark.sql.catalyst.encoders.{Codec, RowEncoder} +import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.TransformingEncoder +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.expressions.{Aggregator, MutableAggregationBuffer, UserDefinedAggregateFunction} +import org.apache.spark.sql.types.StructType + +/** + * An [[Aggregator]] that wraps a [[UserDefinedAggregateFunction]]. This allows us to execute and + * register these (deprecated) UDAFS using the Aggregator code path. + * + * This implementation assumes that the aggregation buffers can be updated in place. See + * `org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate` for more + * information. + */ +private[connect] class UserDefinedAggregateFunctionWrapper(udaf: UserDefinedAggregateFunction) + extends Aggregator[Row, MutableRow, Any] { + override def zero: MutableRow = { + val row = new MutableRow(udaf.bufferSchema) + udaf.initialize(row) + row + } + + override def reduce(b: MutableRow, a: Row): MutableRow = { + udaf.update(b, a) + b + } + + override def merge(b1: MutableRow, b2: MutableRow): MutableRow = { + udaf.merge(b1, b2) + b1 + } + + override def finish(reduction: MutableRow): Any = { + udaf.evaluate(reduction) + } + + override def bufferEncoder: Encoder[MutableRow] = { + TransformingEncoder( + classTag[MutableRow], + RowEncoder.encoderFor(udaf.bufferSchema), + MutableRow) + } + + override def outputEncoder: Encoder[Any] = { + RowEncoder + .encoderForDataType(udaf.dataType, lenient = false) + .asInstanceOf[Encoder[Any]] + } +} + +/** + * Mutable row implementation that is used by [[UserDefinedAggregateFunctionWrapper]]. This code + * assumes that it is allowed to mutate/reuse buffers during aggregation. + */ +private[connect] class MutableRow(private[this] val values: Array[Any]) + extends MutableAggregationBuffer { + def this(schema: StructType) = this(new Array[Any](schema.length)) + def this(row: Row) = this(row.asInstanceOf[GenericRow].values) + override def length: Int = values.length + override def update(i: Int, value: Any): Unit = values(i) = value + override def get(i: Int): Any = values(i) + override def copy(): MutableRow = new MutableRow(values.clone()) + def asGenericRow: Row = new GenericRow(values) +} + +private[connect] object MutableRow extends (() => Codec[MutableRow, Row]) { + object MutableRowCodec extends Codec[MutableRow, Row] { + override def encode(in: MutableRow): Row = in.asGenericRow + override def decode(out: Row): MutableRow = new MutableRow(out) + } + + override def apply(): Codec[MutableRow, Row] = MutableRowCodec +} diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala index f08b6e709f13a..7723dcf131f7f 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/columnNodeSupport.scala @@ -25,10 +25,11 @@ import org.apache.spark.connect.proto.Expression.SortOrder.NullOrdering.{SORT_NU import org.apache.spark.connect.proto.Expression.SortOrder.SortDirection.{SORT_DIRECTION_ASCENDING, SORT_DIRECTION_DESCENDING} import org.apache.spark.connect.proto.Expression.Window.WindowFrame.{FrameBoundary, FrameType} import org.apache.spark.sql.{functions, Column, Encoder} +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin} import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProtoBuilder -import org.apache.spark.sql.expressions.{Aggregator, UserDefinedAggregator, UserDefinedFunction} +import org.apache.spark.sql.expressions.{Aggregator, UserDefinedAggregateFunction, UserDefinedAggregator, UserDefinedFunction} import org.apache.spark.sql.internal.{Alias, CaseWhenOtherwise, Cast, ColumnNode, ColumnNodeLike, InvokeInlineUserDefinedFunction, LambdaFunction, LazyExpression, Literal, SortOrder, SqlExpression, UnresolvedAttribute, UnresolvedExtractValue, UnresolvedFunction, UnresolvedNamedLambdaVariable, UnresolvedRegex, UnresolvedStar, UpdateFields, Window, WindowFrame} /** @@ -150,16 +151,28 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { case InvokeInlineUserDefinedFunction( a: Aggregator[Any @unchecked, Any @unchecked, Any @unchecked], Nil, - false, + isDistinct, _) => // TODO we should probably 'just' detect this particular scenario // in the planner instead of wrapping it in a separate method. - val protoUdf = UdfToProtoUtils.toProto(UserDefinedAggregator(a, e.get)) + val protoUdf = UdfToProtoUtils.toProto(UserDefinedAggregator(a, e.get), Nil, isDistinct) builder.getTypedAggregateExpressionBuilder.setScalarScalaUdf(protoUdf.getScalarScalaUdf) - case InvokeInlineUserDefinedFunction(udf: UserDefinedFunction, args, false, _) => + case InvokeInlineUserDefinedFunction( + udaf: UserDefinedAggregateFunction, + arguments, + isDistinct, + _) => + val wrapped = UserDefinedAggregator( + aggregator = new UserDefinedAggregateFunctionWrapper(udaf), + inputEncoder = RowEncoder.encoderFor(udaf.inputSchema), + deterministic = udaf.deterministic) + builder.setCommonInlineUserDefinedFunction( + UdfToProtoUtils.toProto(wrapped, arguments.map(apply(_, e)), isDistinct)) + + case InvokeInlineUserDefinedFunction(udf: UserDefinedFunction, args, isDistinct, _) => builder.setCommonInlineUserDefinedFunction( - UdfToProtoUtils.toProto(udf, args.map(apply(_, e)))) + UdfToProtoUtils.toProto(udf, args.map(apply(_, e)), isDistinct)) case CaseWhenOtherwise(branches, otherwise, _) => val b = builder.getUnresolvedFunctionBuilder 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 5a89ed5363d0b..6019969be05b9 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 @@ -1814,7 +1814,7 @@ class SparkConnectPlanner( case udf: SparkUserDefinedFunction => UserDefinedFunctionUtils.toScalaUDF(udf, children) case uda: UserDefinedAggregator[_, _, _] => - ScalaAggregator(uda, children).toAggregateExpression() + ScalaAggregator(uda, children).toAggregateExpression(fun.getIsDistinct) case other => throw InvalidPlanInput( s"Unsupported UserDefinedFunction implementation: ${other.getClass}")