Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-14145][SQL] Remove the untyped version of Dataset.groupByKey #11949

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
26 changes: 0 additions & 26 deletions sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1178,32 +1178,6 @@ class Dataset[T] private[sql](
withGroupingKey.newColumns)
}

/**
* :: Experimental ::
* Returns a [[KeyValueGroupedDataset]] where the data is grouped by the given [[Column]]
* expressions.
*
* @group typedrel
* @since 2.0.0
*/
@Experimental
@scala.annotation.varargs
def groupByKey(cols: Column*): KeyValueGroupedDataset[Row, T] = {
val withKeyColumns = logicalPlan.output ++ cols.map(_.expr).map(UnresolvedAlias(_))
val withKey = Project(withKeyColumns, logicalPlan)
val executed = sqlContext.executePlan(withKey)

val dataAttributes = executed.analyzed.output.dropRight(cols.size)
val keyAttributes = executed.analyzed.output.takeRight(cols.size)

new KeyValueGroupedDataset(
RowEncoder(keyAttributes.toStructType),
encoderFor[T],
executed,
dataAttributes,
keyAttributes)
}

/**
* :: Experimental ::
* (Java-specific)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -245,29 +245,6 @@ public Iterator<String> call(Integer key, Iterator<String> left, Iterator<Intege
Assert.assertEquals(asSet("1a#2", "3foobar#6", "5#10"), toSet(cogrouped.collectAsList()));
}

@Test
public void testGroupByColumn() {
List<String> data = Arrays.asList("a", "foo", "bar");
Dataset<String> ds = context.createDataset(data, Encoders.STRING());
KeyValueGroupedDataset<Integer, String> grouped =
ds.groupByKey(length(col("value"))).keyAs(Encoders.INT());

Dataset<String> mapped = grouped.mapGroups(
new MapGroupsFunction<Integer, String, String>() {
@Override
public String call(Integer key, Iterator<String> data) throws Exception {
StringBuilder sb = new StringBuilder(key.toString());
while (data.hasNext()) {
sb.append(data.next());
}
return sb.toString();
}
},
Encoders.STRING());

Assert.assertEquals(asSet("1a", "3foobar"), toSet(mapped.collectAsList()));
}

@Test
public void testSelect() {
List<Integer> data = Arrays.asList(2, 6);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext {

test("persist and then groupBy columns asKey, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupByKey($"_1").keyAs[String]
val grouped = ds.groupByKey(_._1)
val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) }
agged.persist()

Expand Down
49 changes: 0 additions & 49 deletions sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -322,55 +322,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
)
}

test("groupBy columns, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupByKey($"_1")
val agged = grouped.mapGroups { case (g, iter) => (g.getString(0), iter.map(_._2).sum) }

checkDataset(
agged,
("a", 30), ("b", 3), ("c", 1))
}

test("groupBy columns, count") {
val ds = Seq("a" -> 1, "b" -> 1, "a" -> 2).toDS()
val count = ds.groupByKey($"_1").count()

checkDataset(
count,
(Row("a"), 2L), (Row("b"), 1L))
}

test("groupBy columns asKey, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupByKey($"_1").keyAs[String]
val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) }

checkDataset(
agged,
("a", 30), ("b", 3), ("c", 1))
}

test("groupBy columns asKey tuple, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupByKey($"_1", lit(1)).keyAs[(String, Int)]
val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) }

checkDataset(
agged,
(("a", 1), 30), (("b", 1), 3), (("c", 1), 1))
}

test("groupBy columns asKey class, map") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
val grouped = ds.groupByKey($"_1".as("a"), lit(1).as("b")).keyAs[ClassData]
val agged = grouped.mapGroups { case (g, iter) => (g, iter.map(_._2).sum) }

checkDataset(
agged,
(ClassData("a", 1), 30), (ClassData("b", 1), 3), (ClassData("c", 1), 1))
}

test("typed aggregation: expr") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()

Expand Down