Skip to content
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
Original file line number Diff line number Diff line change
Expand Up @@ -473,21 +473,4 @@ class ReplSuite extends SparkFunSuite {
assertDoesNotContain("AssertionError", output)
assertDoesNotContain("Exception", output)
}

test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") {
val resultValue = 12345
val output = runInterpreter("local",
s"""
|val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1)
|val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1))
|val broadcasted = sc.broadcast($resultValue)
|
|// Using broadcast triggers serialization issue in KeyValueGroupedDataset
|val dataset = mapGroups.map(_ => broadcasted.value)
|dataset.collect()
""".stripMargin)
assertDoesNotContain("error:", output)
assertDoesNotContain("Exception", output)
assertContains(s": Array[Int] = Array($resultValue, $resultValue)", output)
}
}
13 changes: 13 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -882,6 +882,19 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
df.withColumn("b", expr("0")).as[ClassData]
.groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() })
}

// This is moved from ReplSuite to prevent java.lang.ClassCircularityError.
test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") {
val resultValue = 12345
val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1)
val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1))
val broadcasted = spark.sparkContext.broadcast(resultValue)

// Using broadcast triggers serialization issue in KeyValueGroupedDataset
val dataset = mapGroups.map(_ => broadcasted.value)

assert(dataset.collect() sameElements Array(resultValue, resultValue))
}
}

case class Generic[T](id: T, value: Double)
Expand Down