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,4 +473,21 @@ 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)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ import org.apache.spark.sql.expressions.ReduceAggregator
class KeyValueGroupedDataset[K, V] private[sql](
kEncoder: Encoder[K],
vEncoder: Encoder[V],
val queryExecution: QueryExecution,
@transient val queryExecution: QueryExecution,
private val dataAttributes: Seq[Attribute],
private val groupingAttributes: Seq[Attribute]) extends Serializable {

Expand Down