diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 469e247addf46..1b8ec881f7c5c 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4356,6 +4356,24 @@ ], "sqlState" : "428EK" }, + "TRANSPOSE_EXCEED_ROW_LIMIT" : { + "message" : [ + "Number of rows exceeds the allowed limit of for TRANSPOSE. If this was intended, set to at least the current row count." + ], + "sqlState" : "54006" + }, + "TRANSPOSE_INVALID_INDEX_COLUMN" : { + "message" : [ + "Invalid index column for TRANSPOSE because: " + ], + "sqlState" : "42804" + }, + "TRANSPOSE_NO_LEAST_COMMON_TYPE" : { + "message" : [ + "Transpose requires non-index columns to share a least common type, but and do not." + ], + "sqlState" : "42K09" + }, "UDTF_ALIAS_NUMBER_MISMATCH" : { "message" : [ "The number of aliases supplied in the AS clause does not match the number of columns output by the UDTF.", diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 552512fee8cd4..f5606215be89d 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -489,6 +489,14 @@ class Dataset[T] private[sql] ( } } + private def buildTranspose(indices: Seq[Column]): DataFrame = + sparkSession.newDataFrame { builder => + val transpose = builder.getTransposeBuilder.setInput(plan.getRoot) + indices.foreach { indexColumn => + transpose.addIndexColumns(indexColumn.expr) + } + } + /** @inheritdoc */ @scala.annotation.varargs def groupBy(cols: Column*): RelationalGroupedDataset = { @@ -582,6 +590,14 @@ class Dataset[T] private[sql] ( buildUnpivot(ids, None, variableColumnName, valueColumnName) } + /** @inheritdoc */ + def transpose(indexColumn: Column): DataFrame = + buildTranspose(Seq(indexColumn)) + + /** @inheritdoc */ + def transpose(): DataFrame = + buildTranspose(Seq.empty) + /** @inheritdoc */ def limit(n: Int): Dataset[T] = sparkSession.newDataset(agnosticEncoder) { builder => builder.getLimitBuilder diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala index 474eac138ab78..ee91f3aa6c00a 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala @@ -552,6 +552,14 @@ class PlanGenerationTestSuite valueColumnName = "value") } + test("transpose index_column") { + simple.transpose(indexColumn = fn.col("id")) + } + + test("transpose no_index_column") { + simple.transpose() + } + test("offset") { simple.offset(1000) } diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index 0e890e3343e66..91b9591625904 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -1849,6 +1849,12 @@ def toArrow(self) -> "pa.Table": def toPandas(self) -> "PandasDataFrameLike": return PandasConversionMixin.toPandas(self) + def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataFrame: + if indexColumn is not None: + return DataFrame(self._jdf.transpose(_to_java_column(indexColumn)), self.sparkSession) + else: + return DataFrame(self._jdf.transpose(), self.sparkSession) + @property def executionInfo(self) -> Optional["ExecutionInfo"]: raise PySparkValueError( diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 442157eef0b75..2f50604ac53b4 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -1858,6 +1858,12 @@ def toPandas(self) -> "PandasDataFrameLike": self._execution_info = ei return pdf + def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataFrame: + return DataFrame( + plan.Transpose(self._plan, [F._to_col(indexColumn)] if indexColumn is not None else []), + self._session, + ) + @property def schema(self) -> StructType: # Schema caching is correct in most cases. Connect is lazy by nature. This means that diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 958626280e41c..fbed0eabc684f 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -1329,6 +1329,27 @@ def plan(self, session: "SparkConnectClient") -> proto.Relation: return plan +class Transpose(LogicalPlan): + """Logical plan object for a transpose operation.""" + + def __init__( + self, + child: Optional["LogicalPlan"], + index_columns: Sequence[Column], + ) -> None: + super().__init__(child) + self.index_columns = index_columns + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + assert self._child is not None + plan = self._create_proto_relation() + plan.transpose.input.CopyFrom(self._child.plan(session)) + if self.index_columns is not None and len(self.index_columns) > 0: + for index_column in self.index_columns: + plan.transpose.index_columns.append(index_column.to_plan(session)) + return plan + + class CollectMetrics(LogicalPlan): """Logical plan object for a CollectMetrics operation.""" diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 9f4d1e717a28d..ee625241600ff 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -36,7 +36,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xe9\x1a\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\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\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"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"\xfb\x04\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\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\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\tdirectionB6\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"\xa3\x1b\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\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\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"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"\xfb\x04\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\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\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\tdirectionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -69,153 +69,155 @@ _PARSE_OPTIONSENTRY._options = None _PARSE_OPTIONSENTRY._serialized_options = b"8\001" _RELATION._serialized_start = 193 - _RELATION._serialized_end = 3626 - _UNKNOWN._serialized_start = 3628 - _UNKNOWN._serialized_end = 3637 - _RELATIONCOMMON._serialized_start = 3640 - _RELATIONCOMMON._serialized_end = 3782 - _SQL._serialized_start = 3785 - _SQL._serialized_end = 4263 - _SQL_ARGSENTRY._serialized_start = 4079 - _SQL_ARGSENTRY._serialized_end = 4169 - _SQL_NAMEDARGUMENTSENTRY._serialized_start = 4171 - _SQL_NAMEDARGUMENTSENTRY._serialized_end = 4263 - _WITHRELATIONS._serialized_start = 4265 - _WITHRELATIONS._serialized_end = 4382 - _READ._serialized_start = 4385 - _READ._serialized_end = 5048 - _READ_NAMEDTABLE._serialized_start = 4563 - _READ_NAMEDTABLE._serialized_end = 4755 - _READ_NAMEDTABLE_OPTIONSENTRY._serialized_start = 4697 - _READ_NAMEDTABLE_OPTIONSENTRY._serialized_end = 4755 - _READ_DATASOURCE._serialized_start = 4758 - _READ_DATASOURCE._serialized_end = 5035 - _READ_DATASOURCE_OPTIONSENTRY._serialized_start = 4697 - _READ_DATASOURCE_OPTIONSENTRY._serialized_end = 4755 - _PROJECT._serialized_start = 5050 - _PROJECT._serialized_end = 5167 - _FILTER._serialized_start = 5169 - _FILTER._serialized_end = 5281 - _JOIN._serialized_start = 5284 - _JOIN._serialized_end = 5945 - _JOIN_JOINDATATYPE._serialized_start = 5623 - _JOIN_JOINDATATYPE._serialized_end = 5715 - _JOIN_JOINTYPE._serialized_start = 5718 - _JOIN_JOINTYPE._serialized_end = 5926 - _SETOPERATION._serialized_start = 5948 - _SETOPERATION._serialized_end = 6427 - _SETOPERATION_SETOPTYPE._serialized_start = 6264 - _SETOPERATION_SETOPTYPE._serialized_end = 6378 - _LIMIT._serialized_start = 6429 - _LIMIT._serialized_end = 6505 - _OFFSET._serialized_start = 6507 - _OFFSET._serialized_end = 6586 - _TAIL._serialized_start = 6588 - _TAIL._serialized_end = 6663 - _AGGREGATE._serialized_start = 6666 - _AGGREGATE._serialized_end = 7432 - _AGGREGATE_PIVOT._serialized_start = 7081 - _AGGREGATE_PIVOT._serialized_end = 7192 - _AGGREGATE_GROUPINGSETS._serialized_start = 7194 - _AGGREGATE_GROUPINGSETS._serialized_end = 7270 - _AGGREGATE_GROUPTYPE._serialized_start = 7273 - _AGGREGATE_GROUPTYPE._serialized_end = 7432 - _SORT._serialized_start = 7435 - _SORT._serialized_end = 7595 - _DROP._serialized_start = 7598 - _DROP._serialized_end = 7739 - _DEDUPLICATE._serialized_start = 7742 - _DEDUPLICATE._serialized_end = 7982 - _LOCALRELATION._serialized_start = 7984 - _LOCALRELATION._serialized_end = 8073 - _CACHEDLOCALRELATION._serialized_start = 8075 - _CACHEDLOCALRELATION._serialized_end = 8147 - _CACHEDREMOTERELATION._serialized_start = 8149 - _CACHEDREMOTERELATION._serialized_end = 8204 - _SAMPLE._serialized_start = 8207 - _SAMPLE._serialized_end = 8480 - _RANGE._serialized_start = 8483 - _RANGE._serialized_end = 8628 - _SUBQUERYALIAS._serialized_start = 8630 - _SUBQUERYALIAS._serialized_end = 8744 - _REPARTITION._serialized_start = 8747 - _REPARTITION._serialized_end = 8889 - _SHOWSTRING._serialized_start = 8892 - _SHOWSTRING._serialized_end = 9034 - _HTMLSTRING._serialized_start = 9036 - _HTMLSTRING._serialized_end = 9150 - _STATSUMMARY._serialized_start = 9152 - _STATSUMMARY._serialized_end = 9244 - _STATDESCRIBE._serialized_start = 9246 - _STATDESCRIBE._serialized_end = 9327 - _STATCROSSTAB._serialized_start = 9329 - _STATCROSSTAB._serialized_end = 9430 - _STATCOV._serialized_start = 9432 - _STATCOV._serialized_end = 9528 - _STATCORR._serialized_start = 9531 - _STATCORR._serialized_end = 9668 - _STATAPPROXQUANTILE._serialized_start = 9671 - _STATAPPROXQUANTILE._serialized_end = 9835 - _STATFREQITEMS._serialized_start = 9837 - _STATFREQITEMS._serialized_end = 9962 - _STATSAMPLEBY._serialized_start = 9965 - _STATSAMPLEBY._serialized_end = 10274 - _STATSAMPLEBY_FRACTION._serialized_start = 10166 - _STATSAMPLEBY_FRACTION._serialized_end = 10265 - _NAFILL._serialized_start = 10277 - _NAFILL._serialized_end = 10411 - _NADROP._serialized_start = 10414 - _NADROP._serialized_end = 10548 - _NAREPLACE._serialized_start = 10551 - _NAREPLACE._serialized_end = 10847 - _NAREPLACE_REPLACEMENT._serialized_start = 10706 - _NAREPLACE_REPLACEMENT._serialized_end = 10847 - _TODF._serialized_start = 10849 - _TODF._serialized_end = 10937 - _WITHCOLUMNSRENAMED._serialized_start = 10940 - _WITHCOLUMNSRENAMED._serialized_end = 11322 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 11184 - _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 11251 - _WITHCOLUMNSRENAMED_RENAME._serialized_start = 11253 - _WITHCOLUMNSRENAMED_RENAME._serialized_end = 11322 - _WITHCOLUMNS._serialized_start = 11324 - _WITHCOLUMNS._serialized_end = 11443 - _WITHWATERMARK._serialized_start = 11446 - _WITHWATERMARK._serialized_end = 11580 - _HINT._serialized_start = 11583 - _HINT._serialized_end = 11715 - _UNPIVOT._serialized_start = 11718 - _UNPIVOT._serialized_end = 12045 - _UNPIVOT_VALUES._serialized_start = 11975 - _UNPIVOT_VALUES._serialized_end = 12034 - _TOSCHEMA._serialized_start = 12047 - _TOSCHEMA._serialized_end = 12153 - _REPARTITIONBYEXPRESSION._serialized_start = 12156 - _REPARTITIONBYEXPRESSION._serialized_end = 12359 - _MAPPARTITIONS._serialized_start = 12362 - _MAPPARTITIONS._serialized_end = 12594 - _GROUPMAP._serialized_start = 12597 - _GROUPMAP._serialized_end = 13232 - _COGROUPMAP._serialized_start = 13235 - _COGROUPMAP._serialized_end = 13761 - _APPLYINPANDASWITHSTATE._serialized_start = 13764 - _APPLYINPANDASWITHSTATE._serialized_end = 14121 - _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_start = 14124 - _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_end = 14368 - _PYTHONUDTF._serialized_start = 14371 - _PYTHONUDTF._serialized_end = 14548 - _COMMONINLINEUSERDEFINEDDATASOURCE._serialized_start = 14551 - _COMMONINLINEUSERDEFINEDDATASOURCE._serialized_end = 14702 - _PYTHONDATASOURCE._serialized_start = 14704 - _PYTHONDATASOURCE._serialized_end = 14779 - _COLLECTMETRICS._serialized_start = 14782 - _COLLECTMETRICS._serialized_end = 14918 - _PARSE._serialized_start = 14921 - _PARSE._serialized_end = 15309 - _PARSE_OPTIONSENTRY._serialized_start = 4697 - _PARSE_OPTIONSENTRY._serialized_end = 4755 - _PARSE_PARSEFORMAT._serialized_start = 15210 - _PARSE_PARSEFORMAT._serialized_end = 15298 - _ASOFJOIN._serialized_start = 15312 - _ASOFJOIN._serialized_end = 15787 + _RELATION._serialized_end = 3684 + _UNKNOWN._serialized_start = 3686 + _UNKNOWN._serialized_end = 3695 + _RELATIONCOMMON._serialized_start = 3698 + _RELATIONCOMMON._serialized_end = 3840 + _SQL._serialized_start = 3843 + _SQL._serialized_end = 4321 + _SQL_ARGSENTRY._serialized_start = 4137 + _SQL_ARGSENTRY._serialized_end = 4227 + _SQL_NAMEDARGUMENTSENTRY._serialized_start = 4229 + _SQL_NAMEDARGUMENTSENTRY._serialized_end = 4321 + _WITHRELATIONS._serialized_start = 4323 + _WITHRELATIONS._serialized_end = 4440 + _READ._serialized_start = 4443 + _READ._serialized_end = 5106 + _READ_NAMEDTABLE._serialized_start = 4621 + _READ_NAMEDTABLE._serialized_end = 4813 + _READ_NAMEDTABLE_OPTIONSENTRY._serialized_start = 4755 + _READ_NAMEDTABLE_OPTIONSENTRY._serialized_end = 4813 + _READ_DATASOURCE._serialized_start = 4816 + _READ_DATASOURCE._serialized_end = 5093 + _READ_DATASOURCE_OPTIONSENTRY._serialized_start = 4755 + _READ_DATASOURCE_OPTIONSENTRY._serialized_end = 4813 + _PROJECT._serialized_start = 5108 + _PROJECT._serialized_end = 5225 + _FILTER._serialized_start = 5227 + _FILTER._serialized_end = 5339 + _JOIN._serialized_start = 5342 + _JOIN._serialized_end = 6003 + _JOIN_JOINDATATYPE._serialized_start = 5681 + _JOIN_JOINDATATYPE._serialized_end = 5773 + _JOIN_JOINTYPE._serialized_start = 5776 + _JOIN_JOINTYPE._serialized_end = 5984 + _SETOPERATION._serialized_start = 6006 + _SETOPERATION._serialized_end = 6485 + _SETOPERATION_SETOPTYPE._serialized_start = 6322 + _SETOPERATION_SETOPTYPE._serialized_end = 6436 + _LIMIT._serialized_start = 6487 + _LIMIT._serialized_end = 6563 + _OFFSET._serialized_start = 6565 + _OFFSET._serialized_end = 6644 + _TAIL._serialized_start = 6646 + _TAIL._serialized_end = 6721 + _AGGREGATE._serialized_start = 6724 + _AGGREGATE._serialized_end = 7490 + _AGGREGATE_PIVOT._serialized_start = 7139 + _AGGREGATE_PIVOT._serialized_end = 7250 + _AGGREGATE_GROUPINGSETS._serialized_start = 7252 + _AGGREGATE_GROUPINGSETS._serialized_end = 7328 + _AGGREGATE_GROUPTYPE._serialized_start = 7331 + _AGGREGATE_GROUPTYPE._serialized_end = 7490 + _SORT._serialized_start = 7493 + _SORT._serialized_end = 7653 + _DROP._serialized_start = 7656 + _DROP._serialized_end = 7797 + _DEDUPLICATE._serialized_start = 7800 + _DEDUPLICATE._serialized_end = 8040 + _LOCALRELATION._serialized_start = 8042 + _LOCALRELATION._serialized_end = 8131 + _CACHEDLOCALRELATION._serialized_start = 8133 + _CACHEDLOCALRELATION._serialized_end = 8205 + _CACHEDREMOTERELATION._serialized_start = 8207 + _CACHEDREMOTERELATION._serialized_end = 8262 + _SAMPLE._serialized_start = 8265 + _SAMPLE._serialized_end = 8538 + _RANGE._serialized_start = 8541 + _RANGE._serialized_end = 8686 + _SUBQUERYALIAS._serialized_start = 8688 + _SUBQUERYALIAS._serialized_end = 8802 + _REPARTITION._serialized_start = 8805 + _REPARTITION._serialized_end = 8947 + _SHOWSTRING._serialized_start = 8950 + _SHOWSTRING._serialized_end = 9092 + _HTMLSTRING._serialized_start = 9094 + _HTMLSTRING._serialized_end = 9208 + _STATSUMMARY._serialized_start = 9210 + _STATSUMMARY._serialized_end = 9302 + _STATDESCRIBE._serialized_start = 9304 + _STATDESCRIBE._serialized_end = 9385 + _STATCROSSTAB._serialized_start = 9387 + _STATCROSSTAB._serialized_end = 9488 + _STATCOV._serialized_start = 9490 + _STATCOV._serialized_end = 9586 + _STATCORR._serialized_start = 9589 + _STATCORR._serialized_end = 9726 + _STATAPPROXQUANTILE._serialized_start = 9729 + _STATAPPROXQUANTILE._serialized_end = 9893 + _STATFREQITEMS._serialized_start = 9895 + _STATFREQITEMS._serialized_end = 10020 + _STATSAMPLEBY._serialized_start = 10023 + _STATSAMPLEBY._serialized_end = 10332 + _STATSAMPLEBY_FRACTION._serialized_start = 10224 + _STATSAMPLEBY_FRACTION._serialized_end = 10323 + _NAFILL._serialized_start = 10335 + _NAFILL._serialized_end = 10469 + _NADROP._serialized_start = 10472 + _NADROP._serialized_end = 10606 + _NAREPLACE._serialized_start = 10609 + _NAREPLACE._serialized_end = 10905 + _NAREPLACE_REPLACEMENT._serialized_start = 10764 + _NAREPLACE_REPLACEMENT._serialized_end = 10905 + _TODF._serialized_start = 10907 + _TODF._serialized_end = 10995 + _WITHCOLUMNSRENAMED._serialized_start = 10998 + _WITHCOLUMNSRENAMED._serialized_end = 11380 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_start = 11242 + _WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY._serialized_end = 11309 + _WITHCOLUMNSRENAMED_RENAME._serialized_start = 11311 + _WITHCOLUMNSRENAMED_RENAME._serialized_end = 11380 + _WITHCOLUMNS._serialized_start = 11382 + _WITHCOLUMNS._serialized_end = 11501 + _WITHWATERMARK._serialized_start = 11504 + _WITHWATERMARK._serialized_end = 11638 + _HINT._serialized_start = 11641 + _HINT._serialized_end = 11773 + _UNPIVOT._serialized_start = 11776 + _UNPIVOT._serialized_end = 12103 + _UNPIVOT_VALUES._serialized_start = 12033 + _UNPIVOT_VALUES._serialized_end = 12092 + _TRANSPOSE._serialized_start = 12105 + _TRANSPOSE._serialized_end = 12227 + _TOSCHEMA._serialized_start = 12229 + _TOSCHEMA._serialized_end = 12335 + _REPARTITIONBYEXPRESSION._serialized_start = 12338 + _REPARTITIONBYEXPRESSION._serialized_end = 12541 + _MAPPARTITIONS._serialized_start = 12544 + _MAPPARTITIONS._serialized_end = 12776 + _GROUPMAP._serialized_start = 12779 + _GROUPMAP._serialized_end = 13414 + _COGROUPMAP._serialized_start = 13417 + _COGROUPMAP._serialized_end = 13943 + _APPLYINPANDASWITHSTATE._serialized_start = 13946 + _APPLYINPANDASWITHSTATE._serialized_end = 14303 + _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_start = 14306 + _COMMONINLINEUSERDEFINEDTABLEFUNCTION._serialized_end = 14550 + _PYTHONUDTF._serialized_start = 14553 + _PYTHONUDTF._serialized_end = 14730 + _COMMONINLINEUSERDEFINEDDATASOURCE._serialized_start = 14733 + _COMMONINLINEUSERDEFINEDDATASOURCE._serialized_end = 14884 + _PYTHONDATASOURCE._serialized_start = 14886 + _PYTHONDATASOURCE._serialized_end = 14961 + _COLLECTMETRICS._serialized_start = 14964 + _COLLECTMETRICS._serialized_end = 15100 + _PARSE._serialized_start = 15103 + _PARSE._serialized_end = 15491 + _PARSE_OPTIONSENTRY._serialized_start = 4755 + _PARSE_OPTIONSENTRY._serialized_end = 4813 + _PARSE_PARSEFORMAT._serialized_start = 15392 + _PARSE_PARSEFORMAT._serialized_end = 15480 + _ASOFJOIN._serialized_start = 15494 + _ASOFJOIN._serialized_end = 15969 # @@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 864803fd33084..b1cd2e184d085 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -104,6 +104,7 @@ class Relation(google.protobuf.message.Message): AS_OF_JOIN_FIELD_NUMBER: builtins.int COMMON_INLINE_USER_DEFINED_DATA_SOURCE_FIELD_NUMBER: builtins.int WITH_RELATIONS_FIELD_NUMBER: builtins.int + TRANSPOSE_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -205,6 +206,8 @@ class Relation(google.protobuf.message.Message): @property def with_relations(self) -> global___WithRelations: ... @property + def transpose(self) -> global___Transpose: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -284,6 +287,7 @@ class Relation(google.protobuf.message.Message): common_inline_user_defined_data_source: global___CommonInlineUserDefinedDataSource | None = ..., with_relations: global___WithRelations | None = ..., + transpose: global___Transpose | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -402,6 +406,8 @@ class Relation(google.protobuf.message.Message): b"to_df", "to_schema", b"to_schema", + "transpose", + b"transpose", "unknown", b"unknown", "unpivot", @@ -519,6 +525,8 @@ class Relation(google.protobuf.message.Message): b"to_df", "to_schema", b"to_schema", + "transpose", + b"transpose", "unknown", b"unknown", "unpivot", @@ -577,6 +585,7 @@ class Relation(google.protobuf.message.Message): "as_of_join", "common_inline_user_defined_data_source", "with_relations", + "transpose", "fill_na", "drop_na", "replace", @@ -3141,6 +3150,47 @@ class Unpivot(google.protobuf.message.Message): global___Unpivot = Unpivot +class Transpose(google.protobuf.message.Message): + """Transpose a DataFrame, switching rows to columns. + Transforms the DataFrame such that the values in the specified index column + become the new columns of the DataFrame. + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + INPUT_FIELD_NUMBER: builtins.int + INDEX_COLUMNS_FIELD_NUMBER: builtins.int + @property + def input(self) -> global___Relation: + """(Required) The input relation.""" + @property + def index_columns( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Optional) A list of columns that will be treated as the indices. + Only single column is supported now. + """ + def __init__( + self, + *, + input: global___Relation | None = ..., + index_columns: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["input", b"input"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal["index_columns", b"index_columns", "input", b"input"], + ) -> None: ... + +global___Transpose = Transpose + class ToSchema(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 7d3900c7afbc5..6bbb715745831 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6311,6 +6311,72 @@ def toPandas(self) -> "PandasDataFrameLike": """ ... + @dispatch_df_method + def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> "DataFrame": + """ + Transposes a DataFrame such that the values in the specified index column become the new + columns of the DataFrame. If no index column is provided, the first column is used as + the default. + + Please note: + - All columns except the index column must share a least common data type. Unless they + are the same data type, all columns are cast to the nearest common data type. + - The name of the column into which the original column names are transposed defaults + to "key". + - null values in the index column are excluded from the column names for the + transposed table, which are ordered in ascending order. + + .. versionadded:: 4.0.0 + + Parameters + ---------- + indexColumn : str or :class:`Column`, optional + The single column that will be treated as the index for the transpose operation. This + column will be used to transform the DataFrame such that the values of the indexColumn + become the new columns in the transposed DataFrame. If not provided, the first column of + the DataFrame will be used as the default. + + Returns + ------- + :class:`DataFrame` + Transposed DataFrame. + + Notes + ----- + Supports Spark Connect. + + Examples + -------- + >>> df = spark.createDataFrame( + ... [("A", 1, 2), ("B", 3, 4)], + ... ["id", "val1", "val2"], + ... ) + >>> df.show() + +---+----+----+ + | id|val1|val2| + +---+----+----+ + | A| 1| 2| + | B| 3| 4| + +---+----+----+ + + >>> df.transpose().show() + +----+---+---+ + | key| A| B| + +----+---+---+ + |val1| 1| 3| + |val2| 2| 4| + +----+---+---+ + + >>> df.transpose(df.id).show() + +----+---+---+ + | key| A| B| + +----+---+---+ + |val1| 1| 3| + |val2| 2| 4| + +----+---+---+ + """ + ... + @property def executionInfo(self) -> Optional["ExecutionInfo"]: """ diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index a214b874f5ec0..8ec0839ec1fe4 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -39,6 +39,7 @@ PySparkTypeError, PySparkValueError, ) +from pyspark.testing import assertDataFrameEqual from pyspark.testing.sqlutils import ( ReusedSQLTestCase, have_pyarrow, @@ -955,6 +956,74 @@ def test_checkpoint_dataframe(self): self.spark.range(1).localCheckpoint().explain() self.assertIn("ExistingRDD", buf.getvalue()) + def test_transpose(self): + df = self.spark.createDataFrame([{"a": "x", "b": "y", "c": "z"}]) + + # default index column + transposed_df = df.transpose() + expected_schema = StructType( + [StructField("key", StringType(), False), StructField("x", StringType(), True)] + ) + expected_data = [Row(key="b", x="y"), Row(key="c", x="z")] + expected_df = self.spark.createDataFrame(expected_data, schema=expected_schema) + assertDataFrameEqual(transposed_df, expected_df, checkRowOrder=True) + + # specified index column + transposed_df = df.transpose("c") + expected_schema = StructType( + [StructField("key", StringType(), False), StructField("z", StringType(), True)] + ) + expected_data = [Row(key="a", z="x"), Row(key="b", z="y")] + expected_df = self.spark.createDataFrame(expected_data, schema=expected_schema) + assertDataFrameEqual(transposed_df, expected_df, checkRowOrder=True) + + # enforce transpose max values + with self.sql_conf({"spark.sql.transposeMaxValues": 0}): + with self.assertRaises(AnalysisException) as pe: + df.transpose().collect() + self.check_error( + exception=pe.exception, + errorClass="TRANSPOSE_EXCEED_ROW_LIMIT", + messageParameters={"maxValues": "0", "config": "spark.sql.transposeMaxValues"}, + ) + + # enforce ascending order based on index column values for transposed columns + df = self.spark.createDataFrame([{"a": "z"}, {"a": "y"}, {"a": "x"}]) + transposed_df = df.transpose() + expected_schema = StructType( + [ + StructField("key", StringType(), False), + StructField("x", StringType(), True), + StructField("y", StringType(), True), + StructField("z", StringType(), True), + ] + ) # z, y, x -> x, y, z + expected_df = self.spark.createDataFrame([], schema=expected_schema) + assertDataFrameEqual(transposed_df, expected_df, checkRowOrder=True) + + # enforce AtomicType Attribute for index column values + df = self.spark.createDataFrame([{"a": ["x", "x"], "b": "y", "c": "z"}]) + with self.assertRaises(AnalysisException) as pe: + df.transpose().collect() + self.check_error( + exception=pe.exception, + errorClass="TRANSPOSE_INVALID_INDEX_COLUMN", + messageParameters={ + "reason": "Index column must be of atomic type, " + "but found: ArrayType(StringType,true)" + }, + ) + + # enforce least common type for non-index columns + df = self.spark.createDataFrame([{"a": "x", "b": "y", "c": 1}]) + with self.assertRaises(AnalysisException) as pe: + df.transpose().collect() + self.check_error( + exception=pe.exception, + errorClass="TRANSPOSE_NO_LEAST_COMMON_TYPE", + messageParameters={"dt1": "STRING", "dt2": "BIGINT"}, + ) + class DataFrameTests(DataFrameTestsMixin, ReusedSQLTestCase): def test_query_execution_unsupported_in_classic(self): diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala index 6af6cc537e476..49237ee7987a1 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala @@ -1508,6 +1508,87 @@ abstract class Dataset[T, DS[U] <: Dataset[U, DS]] extends Serializable { valueColumnName: String): DS[Row] = unpivot(ids, variableColumnName, valueColumnName) + /** + * Transposes a DataFrame such that the values in the specified index column become the new + * columns of the DataFrame. + * + * Please note: + * - All columns except the index column must share a least common data type. Unless they + * are the same data type, all columns are cast to the nearest common data type. + * - The name of the column into which the original column names are transposed defaults + * to "key". + * - null values in the index column are excluded from the column names for the + * transposed table, which are ordered in ascending order. + * + * {{{ + * val df = Seq(("A", 1, 2), ("B", 3, 4)).toDF("id", "val1", "val2") + * df.show() + * // output: + * // +---+----+----+ + * // | id|val1|val2| + * // +---+----+----+ + * // | A| 1| 2| + * // | B| 3| 4| + * // +---+----+----+ + * + * df.transpose($"id").show() + * // output: + * // +----+---+---+ + * // | key| A| B| + * // +----+---+---+ + * // |val1| 1| 3| + * // |val2| 2| 4| + * // +----+---+---+ + * // schema: + * // root + * // |-- key: string (nullable = false) + * // |-- A: integer (nullable = true) + * // |-- B: integer (nullable = true) + * + * df.transpose().show() + * // output: + * // +----+---+---+ + * // | key| A| B| + * // +----+---+---+ + * // |val1| 1| 3| + * // |val2| 2| 4| + * // +----+---+---+ + * // schema: + * // root + * // |-- key: string (nullable = false) + * // |-- A: integer (nullable = true) + * // |-- B: integer (nullable = true) + * }}} + * + * @param indexColumn + * The single column that will be treated as the index for the transpose operation. This column + * will be used to pivot the data, transforming the DataFrame such that the values of the + * indexColumn become the new columns in the transposed DataFrame. + * + * @group untypedrel + * @since 4.0.0 + */ + def transpose(indexColumn: Column): DS[Row] + + /** + * Transposes a DataFrame, switching rows to columns. This function transforms the DataFrame such + * that the values in the first column become the new columns of the DataFrame. + * + * This is equivalent to calling `Dataset#transpose(Column)` where `indexColumn` is set to the + * first column. + * + * Please note: + * - All columns except the index column must share a least common data type. Unless they are + * the same data type, all columns are cast to the nearest common data type. + * - The name of the column into which the original column names are transposed defaults to + * "key". + * - Non-"key" column names for the transposed table are ordered in ascending order. + * + * @group untypedrel + * @since 4.0.0 + */ + def transpose(): DS[Row] + /** * Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset * that returns the same result as the input, with the following guarantees: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 60d979e9c7afb..6f445b1e88d70 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -974,3 +974,13 @@ case object UnresolvedWithinGroup extends LeafExpression with Unevaluable { override def dataType: DataType = throw new UnresolvedException("dataType") override lazy val resolved = false } + +case class UnresolvedTranspose( + indices: Seq[Expression], + child: LogicalPlan +) extends UnresolvedUnaryNode { + final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_TRANSPOSE) + + override protected def withNewChildInternal(newChild: LogicalPlan): UnresolvedTranspose = + copy(child = newChild) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index a2a26924885c0..6ceeeb9bfdf38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -303,7 +303,8 @@ abstract class Optimizer(catalogManager: CatalogManager) ReplaceCurrentLike(catalogManager), SpecialDatetimeValues, RewriteAsOfJoin, - EvalInlineTables + EvalInlineTables, + ReplaceTranspose ) override def apply(plan: LogicalPlan): LogicalPlan = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index 55f222d880844..a524acc19aea8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -184,3 +184,10 @@ object SpecialDatetimeValues extends Rule[LogicalPlan] { } } } + +object ReplaceTranspose extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan.transform { + case t @ Transpose(output, data) => + LocalRelation(output, data) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 398275adf2c12..926027df4c74b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.{AliasIdentifier, SQLConfHelper} +import org.apache.spark.sql.catalyst.{AliasIdentifier, InternalRow, SQLConfHelper} import org.apache.spark.sql.catalyst.analysis.{AnsiTypeCoercion, MultiInstanceRelation, Resolver, TypeCoercion, TypeCoercionBase, UnresolvedUnaryNode} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.catalog.CatalogTable.VIEW_STORING_ANALYZED_PLAN @@ -1474,6 +1474,24 @@ case class Pivot( override protected def withNewChildInternal(newChild: LogicalPlan): Pivot = copy(child = newChild) } +/** + * A logical plan node for transpose, which will later be converted to a [[LocalRelation]] + * at ReplaceTranspose during the query optimization. + * + * The result of the transpose operation is held in the `data` field, and the corresponding + * schema is stored in the `output` field. The `Transpose` node does not depend on any child + * logical plans after the data has been collected and transposed. + * + * @param output A sequence of output attributes representing the schema of the transposed data. + * @param data A sequence of [[InternalRow]] containing the transposed data. + */ +case class Transpose( + output: Seq[Attribute], + data: Seq[InternalRow] = Nil +) extends LeafNode { + final override val nodePatterns: Seq[TreePattern] = Seq(TRANSPOSE) +} + /** * A logical plan node for creating an Unpivot, which will later be converted to an [[Expand]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index 8be7aac7bebf5..c70b43f0db173 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -70,6 +70,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveSubqueryColumnAliases" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTables" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTempViews" :: + "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveTranspose" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUnpivot" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUpCast" :: "org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveUserSpecifiedColumns" :: diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index 6258bd615b440..cbbfccfcab5e8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -92,6 +92,7 @@ object TreePattern extends Enumeration { val SUM: Value = Value val TIME_WINDOW: Value = Value val TIME_ZONE_AWARE_EXPRESSION: Value = Value + val TRANSPOSE: Value = Value val TRUE_OR_FALSE_LITERAL: Value = Value val VARIANT_GET: Value = Value val WINDOW_EXPRESSION: Value = Value @@ -157,6 +158,7 @@ object TreePattern extends Enumeration { val UNRESOLVED_FUNC: Value = Value val UNRESOLVED_SUBQUERY_COLUMN_ALIAS: Value = Value val UNRESOLVED_TABLE_VALUED_FUNCTION: Value = Value + val UNRESOLVED_TRANSPOSE: Value = Value val UNRESOLVED_TVF_ALIASES: Value = Value // Execution expression patterns (alphabetically ordered) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 72915f0e5c256..a87b0613292c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1843,6 +1843,13 @@ object SQLConf { .intConf .createWithDefault(10000) + val DATAFRAME_TRANSPOSE_MAX_VALUES = buildConf("spark.sql.transposeMaxValues") + .doc("When doing a transpose without specifying values for the index column this is" + + " the maximum number of values that will be transposed without error.") + .version("4.0.0") + .intConf + .createWithDefault(500) + val RUN_SQL_ON_FILES = buildConf("spark.sql.runSQLOnFiles") .internal() .doc("When true, we could use `datasource`.`path` as table in SQL query.") @@ -5712,6 +5719,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def dataFramePivotMaxValues: Int = getConf(DATAFRAME_PIVOT_MAX_VALUES) + def dataFrameTransposeMaxValues: Int = getConf(DATAFRAME_TRANSPOSE_MAX_VALUES) + def runSQLonFile: Boolean = getConf(RUN_SQL_ON_FILES) def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP) 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 04fe21086097c..1003e5c21d639 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -76,6 +76,7 @@ message Relation { AsOfJoin as_of_join = 39; CommonInlineUserDefinedDataSource common_inline_user_defined_data_source = 40; WithRelations with_relations = 41; + Transpose transpose = 42; // NA functions NAFill fill_na = 90; @@ -889,6 +890,18 @@ message Unpivot { } } +// Transpose a DataFrame, switching rows to columns. +// Transforms the DataFrame such that the values in the specified index column +// become the new columns of the DataFrame. +message Transpose { + // (Required) The input relation. + Relation input = 1; + + // (Optional) A list of columns that will be treated as the indices. + // Only single column is supported now. + repeated Expression index_columns = 2; +} + message ToSchema { // (Required) The input relation. Relation input = 1; diff --git a/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.json b/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.json new file mode 100644 index 0000000000000..19a2086c8d7df --- /dev/null +++ b/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.json @@ -0,0 +1,20 @@ +{ + "common": { + "planId": "1" + }, + "transpose": { + "input": { + "common": { + "planId": "0" + }, + "localRelation": { + "schema": "struct\u003cid:bigint,a:int,b:double\u003e" + } + }, + "indexColumns": [{ + "unresolvedAttribute": { + "unparsedIdentifier": "id" + } + }] + } +} \ No newline at end of file diff --git a/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.proto.bin new file mode 100644 index 0000000000000..8590932d95cb4 Binary files /dev/null and b/sql/connect/common/src/test/resources/query-tests/queries/transpose_index_column.proto.bin differ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.json b/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.json new file mode 100644 index 0000000000000..82b0448c373e1 --- /dev/null +++ b/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.json @@ -0,0 +1,15 @@ +{ + "common": { + "planId": "1" + }, + "transpose": { + "input": { + "common": { + "planId": "0" + }, + "localRelation": { + "schema": "struct\u003cid:bigint,a:int,b:double\u003e" + } + } + } +} \ No newline at end of file diff --git a/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.proto.bin new file mode 100644 index 0000000000000..c1ea985a64a4b Binary files /dev/null and b/sql/connect/common/src/test/resources/query-tests/queries/transpose_no_index_column.proto.bin differ 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 426e423dd1fc8..b6abab6ef7661 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 @@ -46,7 +46,7 @@ import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, Task import org.apache.spark.sql.{Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.avro.{AvroDataToCatalyst, CatalystDataToAvro} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -203,6 +203,7 @@ class SparkConnectPlanner( transformCachedLocalRelation(rel.getCachedLocalRelation) case proto.Relation.RelTypeCase.HINT => transformHint(rel.getHint) case proto.Relation.RelTypeCase.UNPIVOT => transformUnpivot(rel.getUnpivot) + case proto.Relation.RelTypeCase.TRANSPOSE => transformTranspose(rel.getTranspose) case proto.Relation.RelTypeCase.REPARTITION_BY_EXPRESSION => transformRepartitionByExpression(rel.getRepartitionByExpression) case proto.Relation.RelTypeCase.MAP_PARTITIONS => @@ -1125,6 +1126,13 @@ class SparkConnectPlanner( UnresolvedHint(rel.getName, params, transformRelation(rel.getInput)) } + private def transformTranspose(rel: proto.Transpose): LogicalPlan = { + val child = transformRelation(rel.getInput) + val indices = rel.getIndexColumnsList.asScala.map(transformExpression).toSeq + + UnresolvedTranspose(indices = indices, child = child) + } + private def transformUnpivot(rel: proto.Unpivot): LogicalPlan = { val ids = rel.getIdsList.asScala.toArray.map { expr => column(transformExpression(expr)) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala index 4f6daded402a9..29ad97ad9fbe8 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala @@ -179,6 +179,11 @@ class ProtoToParsedPlanTestSuite logError(log"Skipping ${MDC(PATH, fileName)}") return } + // TODO: enable below by SPARK-49487 + if (fileName.contains("transpose")) { + logError(log"Skipping ${MDC(PATH, fileName)} because of SPARK-49487") + return + } val name = fileName.stripSuffix(".proto.bin") test(name) { val relation = readRelation(file) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 7b6803d41a950..870571b533d09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -981,6 +981,22 @@ class Dataset[T] private[sql]( valueColumnName: String): DataFrame = unpivot(ids.toArray, variableColumnName, valueColumnName) + /** @inheritdoc */ + def transpose(indexColumn: Column): DataFrame = withPlan { + UnresolvedTranspose( + Seq(indexColumn.named), + logicalPlan + ) + } + + /** @inheritdoc */ + def transpose(): DataFrame = withPlan { + UnresolvedTranspose( + Seq.empty, + logicalPlan + ) + } + /** @inheritdoc */ @scala.annotation.varargs def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withTypedPlan { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTranspose.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTranspose.scala new file mode 100644 index 0000000000000..d71237ca15ec3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTranspose.scala @@ -0,0 +1,210 @@ +/* + * 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.catalyst.analysis + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Attribute, AttributeReference, Cast, IsNotNull, Literal, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Limit, LogicalPlan, Project, Sort, Transpose} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{AtomicType, DataType, StringType} +import org.apache.spark.unsafe.types.UTF8String + + +/** + * Rule that resolves and transforms an `UnresolvedTranspose` logical plan into a `Transpose` + * logical plan, which effectively transposes a DataFrame by turning rows into columns based + * on a specified index column. + * + * The high-level logic for the transpose operation is as follows: + * - If the index column is not provided, the first column of the DataFrame is used as the + * default index column. + * - The index column is cast to `StringType` to ensure consistent column naming. + * - Non-index columns are cast to a common data type, determined by finding the least + * common type that can accommodate all non-index columns. + * - The data is sorted by the index column, and rows with `null` index values are excluded + * from the transpose operation. + * - The transposed DataFrame is constructed by turning the original rows into columns, with + * the index column values becoming the new column names and the non-index column values + * populating the transposed data. + */ +class ResolveTranspose(sparkSession: SparkSession) extends Rule[LogicalPlan] { + + private def leastCommonType(dataTypes: Seq[DataType]): DataType = { + if (dataTypes.isEmpty) { + StringType + } else { + dataTypes.reduce { (dt1, dt2) => + AnsiTypeCoercion.findTightestCommonType(dt1, dt2).getOrElse { + throw new AnalysisException( + errorClass = "TRANSPOSE_NO_LEAST_COMMON_TYPE", + messageParameters = Map( + "dt1" -> dt1.sql, + "dt2" -> dt2.sql) + ) + } + } + } + } + + private def transposeMatrix( + fullCollectedRows: Array[InternalRow], + nonIndexColumnNames: Seq[String], + nonIndexColumnDataTypes: Seq[DataType]): Array[Array[Any]] = { + val numTransposedRows = fullCollectedRows.head.numFields - 1 + val numTransposedCols = fullCollectedRows.length + 1 + val finalMatrix = Array.ofDim[Any](numTransposedRows, numTransposedCols) + + // Example of the original DataFrame: + // +---+-----+-----+ + // | id|col1 |col2 | + // +---+-----+-----+ + // | 1| 10 | 20 | + // | 2| 30 | 40 | + // +---+-----+-----+ + // + // After transposition, the finalMatrix will look like: + // [ + // ["col1", 10, 30], // Transposed row for col1 + // ["col2", 20, 40] // Transposed row for col2 + // ] + + for (i <- 0 until numTransposedRows) { + // Insert non-index column name as the first element in each transposed row + finalMatrix(i)(0) = UTF8String.fromString(nonIndexColumnNames(i)) + + for (j <- 1 until numTransposedCols) { + // Insert the transposed data + + // Example: If j = 2, then row = fullCollectedRows(1) + // This corresponds to the second row of the original DataFrame: InternalRow(2, 30, 40) + val row = fullCollectedRows(j - 1) + + // Example: If i = 0 (for "col1"), and j = 2, + // then finalMatrix(0)(2) corresponds to row.get(1, nonIndexColumnDataTypes(0)), + // which accesses the value 30 from InternalRow(2, 30, 40) + finalMatrix(i)(j) = row.get(i + 1, nonIndexColumnDataTypes(i)) + } + } + finalMatrix + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning( + _.containsPattern(TreePattern.UNRESOLVED_TRANSPOSE)) { + case t @ UnresolvedTranspose(indices, child) if child.resolved && indices.forall(_.resolved) => + assert(indices.length == 0 || indices.length == 1, + "The number of index columns should be either 0 or 1.") + + // Handle empty frame with no column headers + if (child.output.isEmpty) { + return Transpose(Seq.empty) + } + + // Use the first column as index column if not provided + val inferredIndexColumn = if (indices.isEmpty) { + child.output.head + } else { + indices.head + } + + // Cast the index column to StringType + val indexColumnAsString = inferredIndexColumn match { + case attr: Attribute if attr.dataType.isInstanceOf[AtomicType] => + Alias(Cast(attr, StringType), attr.name)() + case attr: Attribute => + throw new AnalysisException( + errorClass = "TRANSPOSE_INVALID_INDEX_COLUMN", + messageParameters = Map( + "reason" -> s"Index column must be of atomic type, but found: ${attr.dataType}") + ) + case _ => + throw new AnalysisException( + errorClass = "TRANSPOSE_INVALID_INDEX_COLUMN", + messageParameters = Map( + "reason" -> s"Index column must be an atomic attribute") + ) + } + + // Cast non-index columns to the least common type + val nonIndexColumns = child.output.filterNot( + _.exprId == inferredIndexColumn.asInstanceOf[Attribute].exprId) + val nonIndexTypes = nonIndexColumns.map(_.dataType) + val commonType = leastCommonType(nonIndexTypes) + val nonIndexColumnsAsLCT = nonIndexColumns.map { attr => + Alias(Cast(attr, commonType), attr.name)() + } + + // Exclude nulls and sort index column values, and collect the casted frame + val allCastCols = indexColumnAsString +: nonIndexColumnsAsLCT + val nonNullChild = Filter(IsNotNull(inferredIndexColumn), child) + val sortedChild = Sort( + Seq(SortOrder(inferredIndexColumn, Ascending)), + global = true, + nonNullChild + ) + val projectAllCastCols = Project(allCastCols, sortedChild) + val maxValues = sparkSession.sessionState.conf.dataFrameTransposeMaxValues + val limit = Literal(maxValues + 1) + val limitedProject = Limit(limit, projectAllCastCols) + val queryExecution = sparkSession.sessionState.executePlan(limitedProject) + val fullCollectedRows = queryExecution.executedPlan.executeCollect() + + if (fullCollectedRows.isEmpty) { + // Return a DataFrame with a single column "key" containing non-index column names + val keyAttr = AttributeReference("key", StringType, nullable = false)() + val keyValues = nonIndexColumns.map(col => UTF8String.fromString(col.name)) + val keyRows = keyValues.map(value => InternalRow(value)) + + Transpose(Seq(keyAttr), keyRows) + } else { + if (fullCollectedRows.length > maxValues) { + throw new AnalysisException( + errorClass = "TRANSPOSE_EXCEED_ROW_LIMIT", + messageParameters = Map( + "maxValues" -> maxValues.toString, + "config" -> SQLConf.DATAFRAME_TRANSPOSE_MAX_VALUES.key)) + } + + // Transpose the matrix + val nonIndexColumnNames = nonIndexColumns.map(_.name) + val nonIndexColumnDataTypes = projectAllCastCols.output.tail.map(attr => attr.dataType) + val transposedMatrix = transposeMatrix( + fullCollectedRows, nonIndexColumnNames, nonIndexColumnDataTypes) + val transposedInternalRows = transposedMatrix.map { row => + InternalRow.fromSeq(row.toIndexedSeq) + } + + // Construct output attributes + val keyAttr = AttributeReference("key", StringType, nullable = false)() + val transposedColumnNames = fullCollectedRows.map { row => row.getString(0) } + val valueAttrs = transposedColumnNames.map { value => + AttributeReference( + value, + commonType + )() + } + + val transposeOutput = (keyAttr +: valueAttrs).toIndexedSeq + val transposeData = transposedInternalRows.toIndexedSeq + Transpose(transposeOutput, transposeData) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 99287bddb5104..a2539828733fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.internal import org.apache.spark.annotation.Unstable import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} import org.apache.spark.sql.artifact.ArtifactManager -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, FunctionRegistry, ReplaceCharWithVarchar, ResolveSessionCatalog, TableFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, FunctionRegistry, ReplaceCharWithVarchar, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.optimizer.Optimizer @@ -205,6 +205,7 @@ abstract class BaseSessionStateBuilder( new ResolveSessionCatalog(this.catalogManager) +: ResolveWriteToStream +: new EvalSubqueriesForTimeTravel +: + new ResolveTranspose(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala new file mode 100644 index 0000000000000..e6e8b6d5e5b01 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTransposeSuite.scala @@ -0,0 +1,175 @@ +/* + * 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 + +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types._ + +class DataFrameTransposeSuite extends QueryTest with SharedSparkSession { + import testImplicits._ + + // + // Test cases: input parameter + // + + test("transpose with default index column") { + checkAnswer( + salary.transpose(), + Row("salary", 2000.0, 1000.0) :: Nil + ) + } + + test("transpose with specified index column") { + checkAnswer( + salary.transpose($"salary"), + Row("personId", 1, 0) :: Nil + ) + } + + // + // Test cases: API behavior + // + + test("enforce least common type for non-index columns") { + val df = Seq(("x", 1, 10.0), ("y", 2, 20.0)).toDF("name", "id", "value") + val transposedDf = df.transpose() + checkAnswer( + transposedDf, + Row("id", 1.0, 2.0) :: Row("value", 10.0, 20.0) :: Nil + ) + // (id,IntegerType) -> (x,DoubleType) + // (value,DoubleType) -> (y,DoubleType) + assertResult(DoubleType)(transposedDf.schema("x").dataType) + assertResult(DoubleType)(transposedDf.schema("y").dataType) + + val exception = intercept[AnalysisException] { + person.transpose() + } + assert(exception.getMessage.contains( + "[TRANSPOSE_NO_LEAST_COMMON_TYPE] Transpose requires non-index columns " + + "to share a least common type")) + } + + test("enforce ascending order based on index column values for transposed columns") { + val transposedDf = person.transpose($"name") + checkAnswer( + transposedDf, + Row("id", 1, 0) :: Row("age", 20, 30) :: Nil + ) + // mike, jim -> jim, mike + assertResult(Array("key", "jim", "mike"))(transposedDf.columns) + } + + test("enforce AtomicType Attribute for index column values") { + val exceptionAtomic = intercept[AnalysisException] { + complexData.transpose($"m") // (m,MapType(StringType,IntegerType,false)) + } + assert(exceptionAtomic.getMessage.contains( + "[TRANSPOSE_INVALID_INDEX_COLUMN] Invalid index column for TRANSPOSE because:" + + " Index column must be of atomic type, but found")) + + val exceptionAttribute = intercept[AnalysisException] { + // (s,StructType(StructField(key,IntegerType,false),StructField(value,StringType,true))) + complexData.transpose($"s.key") + } + assert(exceptionAttribute.getMessage.contains( + "[TRANSPOSE_INVALID_INDEX_COLUMN] Invalid index column for TRANSPOSE because:" + + " Index column must be an atomic attribute")) + } + + test("enforce transpose max values") { + spark.conf.set(SQLConf.DATAFRAME_TRANSPOSE_MAX_VALUES.key, 1) + val exception = intercept[AnalysisException]( + person.transpose($"name") + ) + assert(exception.getMessage.contains( + "[TRANSPOSE_EXCEED_ROW_LIMIT] Number of rows exceeds the allowed limit of")) + spark.conf.set(SQLConf.DATAFRAME_TRANSPOSE_MAX_VALUES.key, + SQLConf.DATAFRAME_TRANSPOSE_MAX_VALUES.defaultValue.get) + } + + // + // Test cases: special frame + // + + test("transpose empty frame w. column names") { + val schema = StructType(Seq( + StructField("id", IntegerType), + StructField("name", StringType) + )) + val emptyDF = spark.createDataFrame(spark.sparkContext.emptyRDD[Row], schema) + val transposedDF = emptyDF.transpose() + checkAnswer( + transposedDF, + Row("name") :: Nil + ) + assertResult(StringType)(transposedDF.schema("key").dataType) + } + + test("transpose empty frame w/o column names") { + val emptyDF = spark.emptyDataFrame + checkAnswer( + emptyDF.transpose(), + Nil + ) + } + + test("transpose frame with only index column") { + val transposedDf = tableName.transpose() + checkAnswer( + transposedDf, + Nil + ) + assertResult(Array("key", "test"))(transposedDf.columns) + } + + test("transpose frame with duplicates in index column") { + val df = Seq( + ("A", 1, 2), + ("B", 3, 4), + ("A", 5, 6) + ).toDF("id", "val1", "val2") + val transposedDf = df.transpose() + checkAnswer( + transposedDf, + Seq( + Row("val1", 1, 5, 3), + Row("val2", 2, 6, 4) + ) + ) + assertResult(Array("key", "A", "A", "B"))(transposedDf.columns) + } + + test("transpose frame with nulls in index column") { + val df = Seq( + ("A", 1, 2), + ("B", 3, 4), + (null, 5, 6) + ).toDF("id", "val1", "val2") + val transposedDf = df.transpose() + checkAnswer( + transposedDf, + Seq( + Row("val1", 1, 3), + Row("val2", 2, 4) + ) + ) + assertResult(Array("key", "A", "B"))(transposedDf.columns) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index c455c2cef15fd..44c1ecd6902ce 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ReplaceCharWithVarchar, ResolveSessionCatalog} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ReplaceCharWithVarchar, ResolveSessionCatalog, ResolveTranspose} import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -94,6 +94,7 @@ class HiveSessionStateBuilder( ResolveWriteToStream +: new EvalSubqueriesForTimeTravel +: new DetermineTableStats(session) +: + new ResolveTranspose(session) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] =