From 0f7c9e84e0d00813bf56712097677add5657f19f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 4 Nov 2016 23:34:29 -0700 Subject: [PATCH] [SPARK-18189] [SQL] [Followup] Move test from ReplSuite to prevent java.lang.ClassCircularityError closes #15774 --- .../scala/org/apache/spark/repl/ReplSuite.scala | 17 ----------------- .../org/apache/spark/sql/DatasetSuite.scala | 12 ++++++++++++ 2 files changed, 12 insertions(+), 17 deletions(-) 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 96d2dfc265..9262e938c2 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 55f0487805..6fa7b04877 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 @@ -923,6 +923,18 @@ class DatasetSuite extends QueryTest with SharedSQLContext { .groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() }) } + 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)) + } + Seq(true, false).foreach { eager => def testCheckpointing(testName: String)(f: => Unit): Unit = { test(s"Dataset.checkpoint() - $testName (eager = $eager)") {