Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -67,11 +67,27 @@ case class ReferenceToExpressions(result: Expression, children: Seq[Expression])
case (childGen, child) => LambdaVariable(childGen.value, childGen.isNull, child.dataType)
}

// SPARK-18125: The children vars are local variables. If the result expression uses
// splitExpression, those variables cannot be accessed so compilation fails.
// To fix it, we use class variables to hold those local variables.
val initClassChildVars = childrenVars.map { childVar =>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we merge this with childrenVars?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We could. Not sure if merging them will reduce readability or not.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let me refactor it a bit.

val childVarInClass = ctx.freshName("childVarInClass")
ctx.addMutableState(ctx.javaType(childVar.dataType), childVarInClass, "")
val isNullInClass = ctx.freshName("childVarInClassIsNull")
ctx.addMutableState("boolean", isNullInClass, "")
LambdaVariable(childVarInClass, isNullInClass, childVar.dataType)
}

val initClassChildVarsCode = initClassChildVars.zipWithIndex.map { case (childVarInClass, i) =>
s"${childVarInClass.value} = ${childrenVars(i).value};\n" +
s"${childVarInClass.isNull} = ${childrenVars(i).isNull};"
}.mkString("\n")

val resultGen = result.transform {
case b: BoundReference => childrenVars(b.ordinal)
case b: BoundReference => initClassChildVars(b.ordinal)
}.genCode(ctx)

ExprCode(code = childrenGen.map(_.code).mkString("\n") + "\n" + resultGen.code,
isNull = resultGen.isNull, value = resultGen.value)
ExprCode(code = childrenGen.map(_.code).mkString("\n") + "\n" + initClassChildVarsCode +
resultGen.code, isNull = resultGen.isNull, value = resultGen.value)
}
}
37 changes: 37 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 @@ -919,6 +919,40 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
df.withColumn("b", expr("0")).as[ClassData]
.groupByKey(_.a).flatMapGroups { case (x, iter) => List[Int]() })
}

test("SPARK-18125: Spark generated code causes CompileException") {
val data = Array(
Route("a", "b", 1),
Route("a", "b", 2),
Route("a", "c", 2),
Route("a", "d", 10),
Route("b", "a", 1),
Route("b", "a", 5),
Route("b", "c", 6))
val ds = sparkContext.parallelize(data).toDF.as[Route]

val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
.groupByKey(r => (r.src, r.dest))
.reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) =>
GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes)
}.map(_._2)

val expected = Seq(
GroupedRoutes("a", "d", Seq(Route("a", "d", 10))),
GroupedRoutes("b", "c", Seq(Route("b", "c", 6))),
GroupedRoutes("a", "b", Seq(Route("a", "b", 1), Route("a", "b", 2))),
GroupedRoutes("b", "a", Seq(Route("b", "a", 1), Route("b", "a", 5))),
GroupedRoutes("a", "c", Seq(Route("a", "c", 2)))
)

implicit def ordering[GroupedRoutes]: Ordering[GroupedRoutes] = new Ordering[GroupedRoutes] {
override def compare(x: GroupedRoutes, y: GroupedRoutes): Int = {
x.toString.compareTo(y.toString)
}
}

checkDatasetUnorderly(grped, expected: _*)
}
}

case class Generic[T](id: T, value: Double)
Expand Down Expand Up @@ -991,3 +1025,6 @@ object DatasetTransform {
ds.map(_ + 1)
}
}

case class Route(src: String, dest: String, cost: Int)
case class GroupedRoutes(src: String, dest: String, routes: Seq[Route])