Skip to content

Commit 8b99e20

Browse files
seyferxin
authored andcommitted
[SPARK-18189][SQL][FOLLOWUP] Move test from ReplSuite to prevent java.lang.ClassCircularityError
## What changes were proposed in this pull request? Move the test which is causing java.lang.ClassCircularityError from ReplSuite to DatasetSuite. ## How was this patch tested? > build/mvn -DskipTests -Phadoop-2.3 -Pyarn -Phive -Phive-thriftserver -Pkinesis-asl -Pmesos clean package > build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.repl.ReplSuite test Author: Ergin Seyfe <[email protected]> Closes #15774 from seyfe/fix_replsuite_test_error_branch2.0.
1 parent 399597b commit 8b99e20

File tree

2 files changed

+13
-17
lines changed

2 files changed

+13
-17
lines changed

repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -473,21 +473,4 @@ class ReplSuite extends SparkFunSuite {
473473
assertDoesNotContain("AssertionError", output)
474474
assertDoesNotContain("Exception", output)
475475
}
476-
477-
test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") {
478-
val resultValue = 12345
479-
val output = runInterpreter("local",
480-
s"""
481-
|val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1)
482-
|val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1))
483-
|val broadcasted = sc.broadcast($resultValue)
484-
|
485-
|// Using broadcast triggers serialization issue in KeyValueGroupedDataset
486-
|val dataset = mapGroups.map(_ => broadcasted.value)
487-
|dataset.collect()
488-
""".stripMargin)
489-
assertDoesNotContain("error:", output)
490-
assertDoesNotContain("Exception", output)
491-
assertContains(s": Array[Int] = Array($resultValue, $resultValue)", output)
492-
}
493476
}

sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -882,6 +882,19 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
882882
df.withColumn("b", expr("0")).as[ClassData]
883883
.groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() })
884884
}
885+
886+
// This is moved from ReplSuite to prevent java.lang.ClassCircularityError.
887+
test("SPARK-18189: Fix serialization issue in KeyValueGroupedDataset") {
888+
val resultValue = 12345
889+
val keyValueGrouped = Seq((1, 2), (3, 4)).toDS().groupByKey(_._1)
890+
val mapGroups = keyValueGrouped.mapGroups((k, v) => (k, 1))
891+
val broadcasted = spark.sparkContext.broadcast(resultValue)
892+
893+
// Using broadcast triggers serialization issue in KeyValueGroupedDataset
894+
val dataset = mapGroups.map(_ => broadcasted.value)
895+
896+
assert(dataset.collect() sameElements Array(resultValue, resultValue))
897+
}
885898
}
886899

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

0 commit comments

Comments
 (0)