diff --git a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 8deafe3bcd962..f7d7a4f041315 100644 --- a/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -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) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f897cfb26d3ff..6113e5d33b407 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -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)