@@ -62,7 +62,7 @@ import org.apache.spark.sql.execution.datasources.LogicalRelation
6262import org .apache .spark .sql .execution .datasources .v2 .{DataSourceV2Relation , DataSourceV2ScanRelation , FileTable }
6363import org .apache .spark .sql .execution .python .EvaluatePython
6464import org .apache .spark .sql .execution .stat .StatFunctions
65- import org .apache .spark .sql .internal .{DataFrameWriterImpl , DataFrameWriterV2Impl , MergeIntoWriterImpl , SQLConf }
65+ import org .apache .spark .sql .internal .{DataFrameWriterImpl , DataFrameWriterV2Impl , MergeIntoWriterImpl , SQLConf , ToScalaUDF }
6666import org .apache .spark .sql .internal .ExpressionUtils .column
6767import org .apache .spark .sql .internal .TypedAggUtils .withInputType
6868import org .apache .spark .sql .streaming .DataStreamWriter
@@ -865,7 +865,24 @@ class Dataset[T] private[sql](
865865 Filter (condition.expr, logicalPlan)
866866 }
867867
868- /** @inheritdoc */
868+ /**
869+ * Groups the Dataset using the specified columns, so we can run aggregation on them. See
870+ * [[RelationalGroupedDataset ]] for all the available aggregate functions.
871+ *
872+ * {{{
873+ * // Compute the average for all numeric columns grouped by department.
874+ * ds.groupBy($"department").avg()
875+ *
876+ * // Compute the max age and average salary, grouped by department and gender.
877+ * ds.groupBy($"department", $"gender").agg(Map(
878+ * "salary" -> "avg",
879+ * "age" -> "max"
880+ * ))
881+ * }}}
882+ *
883+ * @group untypedrel
884+ * @since 2.0.0
885+ */
869886 @ scala.annotation.varargs
870887 def groupBy (cols : Column * ): RelationalGroupedDataset = {
871888 RelationalGroupedDataset (toDF(), cols.map(_.expr), RelationalGroupedDataset .GroupByType )
@@ -897,7 +914,13 @@ class Dataset[T] private[sql](
897914 rdd.reduce(func)
898915 }
899916
900- /** @inheritdoc */
917+ /**
918+ * (Scala-specific)
919+ * Returns a [[KeyValueGroupedDataset ]] where the data is grouped by the given key `func`.
920+ *
921+ * @group typedrel
922+ * @since 2.0.0
923+ */
901924 def groupByKey [K : Encoder ](func : T => K ): KeyValueGroupedDataset [K , T ] = {
902925 val withGroupingKey = AppendColumns (func, logicalPlan)
903926 val executed = sparkSession.sessionState.executePlan(withGroupingKey)
@@ -910,6 +933,16 @@ class Dataset[T] private[sql](
910933 withGroupingKey.newColumns)
911934 }
912935
936+ /**
937+ * (Java-specific)
938+ * Returns a [[KeyValueGroupedDataset ]] where the data is grouped by the given key `func`.
939+ *
940+ * @group typedrel
941+ * @since 2.0.0
942+ */
943+ def groupByKey [K ](func : MapFunction [T , K ], encoder : Encoder [K ]): KeyValueGroupedDataset [K , T ] =
944+ groupByKey(ToScalaUDF (func))(encoder)
945+
913946 /** @inheritdoc */
914947 def unpivot (
915948 ids : Array [Column ],
@@ -1607,7 +1640,28 @@ class Dataset[T] private[sql](
16071640 new DataFrameWriterV2Impl [T ](table, this )
16081641 }
16091642
1610- /** @inheritdoc */
1643+ /**
1644+ * Merges a set of updates, insertions, and deletions based on a source table into
1645+ * a target table.
1646+ *
1647+ * Scala Examples:
1648+ * {{{
1649+ * spark.table("source")
1650+ * .mergeInto("target", $"source.id" === $"target.id")
1651+ * .whenMatched($"salary" === 100)
1652+ * .delete()
1653+ * .whenNotMatched()
1654+ * .insertAll()
1655+ * .whenNotMatchedBySource($"salary" === 100)
1656+ * .update(Map(
1657+ * "salary" -> lit(200)
1658+ * ))
1659+ * .merge()
1660+ * }}}
1661+ *
1662+ * @group basic
1663+ * @since 4.0.0
1664+ */
16111665 def mergeInto (table : String , condition : Column ): MergeIntoWriter [T ] = {
16121666 if (isStreaming) {
16131667 logicalPlan.failAnalysis(
@@ -1970,12 +2024,6 @@ class Dataset[T] private[sql](
19702024 @ scala.annotation.varargs
19712025 override def agg (expr : Column , exprs : Column * ): DataFrame = super .agg(expr, exprs : _* )
19722026
1973- /** @inheritdoc */
1974- override def groupByKey [K ](
1975- func : MapFunction [T , K ],
1976- encoder : Encoder [K ]): KeyValueGroupedDataset [K , T ] =
1977- super .groupByKey(func, encoder).asInstanceOf [KeyValueGroupedDataset [K , T ]]
1978-
19792027 // //////////////////////////////////////////////////////////////////////////
19802028 // For Python API
19812029 // //////////////////////////////////////////////////////////////////////////
0 commit comments