From 942dad7707aa250de55dfe4d873400cb0418dcdd Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 9 Nov 2015 17:48:23 +0800 Subject: [PATCH 01/25] Replace catalyst converter with RowEncoder. --- .../sql/catalyst/expressions/ScalaUDF.scala | 1167 ++++------------- 1 file changed, 250 insertions(+), 917 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 3388cc20a980..18cc2a63ec19 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.Row +import org.apache.spark.sql.types._ /** * User-defined function. @@ -37,21 +39,36 @@ case class ScalaUDF( override def toString: String = s"UDF(${children.mkString(",")})" + // Accessors used in genCode + def userDefinedFunc(): AnyRef = function + def getChildren(): Seq[Expression] = children + + val inputSchema: StructType = { + val fields = if (inputTypes == Nil) { + // from the deprecated callUDF codepath + children.zipWithIndex.map { case (e, i) => + StructField(s"_c$i", e.dataType) + } + } else { + inputTypes.zipWithIndex.map { case (t, i) => + StructField(s"_c$i", t) + } + } + StructType(fields) + } + // scalastyle:off /** This method has been generated by this script (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) - val converters = (0 to x - 1).map(x => s"lazy val converter$x = CatalystTypeConverters.createToScalaConverter(child$x.dataType)").reduce(_ + "\n " + _) - val evals = (0 to x - 1).map(x => s"converter$x(child$x.eval(input))").reduce(_ + ",\n " + _) + val evals = (0 to x - 1).map(x => s"convertedRow.get($x)").reduce(_ + ",\n " + _) s"""case $x => val func = function.asInstanceOf[($anys) => Any] - $childs - $converters (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) func( $evals) } @@ -60,924 +77,238 @@ case class ScalaUDF( */ - // Accessors used in genCode - def userDefinedFunc(): AnyRef = function - def getChildren(): Seq[Expression] = children - - private[this] val f = children.size match { - case 0 => - val func = function.asInstanceOf[() => Any] - (input: InternalRow) => { - func() - } - - case 1 => - val func = function.asInstanceOf[(Any) => Any] - val child0 = children(0) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input))) - } - - case 2 => - val func = function.asInstanceOf[(Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input))) - } - - case 3 => - val func = function.asInstanceOf[(Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input))) - } - - case 4 => - val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input))) - } - - case 5 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input))) - } - - case 6 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input))) - } - - case 7 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input))) - } - - case 8 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input))) - } - - case 9 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input))) - } - - case 10 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input))) - } - - case 11 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input))) - } - - case 12 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input))) - } - - case 13 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input))) - } - - case 14 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input))) - } - - case 15 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input))) - } - - case 16 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input))) - } - - case 17 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input))) - } - - case 18 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - val child17 = children(17) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input)), - converter17(child17.eval(input))) - } - - case 19 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - val child17 = children(17) - val child18 = children(18) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input)), - converter17(child17.eval(input)), - converter18(child18.eval(input))) - } - - case 20 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - val child17 = children(17) - val child18 = children(18) - val child19 = children(19) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input)), - converter17(child17.eval(input)), - converter18(child18.eval(input)), - converter19(child19.eval(input))) - } - - case 21 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - val child17 = children(17) - val child18 = children(18) - val child19 = children(19) - val child20 = children(20) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input)), - converter17(child17.eval(input)), - converter18(child18.eval(input)), - converter19(child19.eval(input)), - converter20(child20.eval(input))) - } - - case 22 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - val child0 = children(0) - val child1 = children(1) - val child2 = children(2) - val child3 = children(3) - val child4 = children(4) - val child5 = children(5) - val child6 = children(6) - val child7 = children(7) - val child8 = children(8) - val child9 = children(9) - val child10 = children(10) - val child11 = children(11) - val child12 = children(12) - val child13 = children(13) - val child14 = children(14) - val child15 = children(15) - val child16 = children(16) - val child17 = children(17) - val child18 = children(18) - val child19 = children(19) - val child20 = children(20) - val child21 = children(21) - lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) - lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) - (input: InternalRow) => { - func( - converter0(child0.eval(input)), - converter1(child1.eval(input)), - converter2(child2.eval(input)), - converter3(child3.eval(input)), - converter4(child4.eval(input)), - converter5(child5.eval(input)), - converter6(child6.eval(input)), - converter7(child7.eval(input)), - converter8(child8.eval(input)), - converter9(child9.eval(input)), - converter10(child10.eval(input)), - converter11(child11.eval(input)), - converter12(child12.eval(input)), - converter13(child13.eval(input)), - converter14(child14.eval(input)), - converter15(child15.eval(input)), - converter16(child16.eval(input)), - converter17(child17.eval(input)), - converter18(child18.eval(input)), - converter19(child19.eval(input)), - converter20(child20.eval(input)), - converter21(child21.eval(input))) - } + private[this] val f = { + lazy val inputEncoder: ExpressionEncoder[Row] = RowEncoder(inputSchema) + children.size match { + case 0 => + val func = function.asInstanceOf[() => Any] + (input: InternalRow) => { + func() + } + + case 1 => + val func = function.asInstanceOf[(Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0)) + } + + case 2 => + val func = function.asInstanceOf[(Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1)) + } + + case 3 => + val func = function.asInstanceOf[(Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2)) + } + + case 4 => + val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3)) + } + + case 5 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4)) + } + + case 6 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5)) + } + + case 7 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6)) + } + + case 8 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7)) + } + + case 9 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8)) + } + + case 10 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9)) + } + + case 11 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10)) + } + + case 12 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11)) + } + + case 13 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12)) + } + + case 14 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13)) + } + + case 15 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14)) + } + + case 16 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15)) + } + + case 17 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16)) + } + + case 18 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16), convertedRow.get(17)) + } + + case 19 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16), convertedRow.get(17), convertedRow.get(18)) + } + + case 20 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19)) + } + + case 21 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19), + convertedRow.get(20)) + } + + case 22 => + val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] + (input: InternalRow) => { + val convertedRow: Row = inputEncoder.fromRow(input) + func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), + convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), + convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), + convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), + convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19), + convertedRow.get(20), convertedRow.get(21)) + } + } } // scalastyle:on // Generate codes used to convert the arguments to Scala type for user-defined funtions - private[this] def genCodeForConverter(ctx: CodeGenContext, index: Int): String = { + private[this] def genCodeForConverter( + ctx: CodeGenContext, + scalaUDFTermIdx: Int, + index: Int): String = { val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" val expressionClassName = classOf[Expression].getName val scalaUDFClassName = classOf[ScalaUDF].getName val converterTerm = ctx.freshName("converter") - val expressionIdx = ctx.references.size - 1 ctx.addMutableState(converterClassName, converterTerm, s"this.$converterTerm = ($converterClassName)$typeConvertersClassName" + s".createToScalaConverter(((${expressionClassName})((($scalaUDFClassName)" + - s"expressions[$expressionIdx]).getChildren().apply($index))).dataType());") + s"expressions[$scalaUDFTermIdx]).getChildren().apply($index))).dataType());") converterTerm } @@ -986,6 +317,7 @@ case class ScalaUDF( ev: GeneratedExpressionCode): String = { ctx.references += this + val scalaUDFTermIdx = ctx.references.size - 1 val scalaUDFClassName = classOf[ScalaUDF].getName val converterClassName = classOf[Any => Any].getName @@ -994,26 +326,24 @@ case class ScalaUDF( // Generate codes used to convert the returned value of user-defined functions to Catalyst type val catalystConverterTerm = ctx.freshName("catalystConverter") - val catalystConverterTermIdx = ctx.references.size - 1 ctx.addMutableState(converterClassName, catalystConverterTerm, s"this.$catalystConverterTerm = ($converterClassName)$typeConvertersClassName" + s".createToCatalystConverter((($scalaUDFClassName)expressions" + - s"[$catalystConverterTermIdx]).dataType());") + s"[$scalaUDFTermIdx]).dataType());") val resultTerm = ctx.freshName("result") // This must be called before children expressions' codegen // because ctx.references is used in genCodeForConverter - val converterTerms = (0 until children.size).map(genCodeForConverter(ctx, _)) + val converterTerms = (0 until children.size).map(genCodeForConverter(ctx, scalaUDFTermIdx, _)) // Initialize user-defined function val funcClassName = s"scala.Function${children.size}" val funcTerm = ctx.freshName("udf") - val funcExpressionIdx = ctx.references.size - 1 ctx.addMutableState(funcClassName, funcTerm, s"this.$funcTerm = ($funcClassName)((($scalaUDFClassName)expressions" + - s"[$funcExpressionIdx]).userDefinedFunc());") + s"[$scalaUDFTermIdx]).userDefinedFunc());") // codegen for children expressions val evals = children.map(_.gen(ctx)) @@ -1041,6 +371,9 @@ case class ScalaUDF( """ } - private[this] val converter = CatalystTypeConverters.createToCatalystConverter(dataType) - override def eval(input: InternalRow): Any = converter(f(input)) + override def eval(input: InternalRow): Any = { + val outputEncoder: ExpressionEncoder[Row] = + RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) + outputEncoder.toRow(Row(f(input))) + } } From 39f6c26bce822d1a7cb1a5174f5da0c65cf9977b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Nov 2015 01:27:34 +0800 Subject: [PATCH 02/25] Add UserDefinedType to RowEncoder. --- .../spark/sql/catalyst/encoders/RowEncoder.scala | 12 ++++++++++++ .../spark/sql/catalyst/expressions/ScalaUDF.scala | 5 +++-- 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 0b42130a013b..a8af11b5b45f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -50,6 +50,9 @@ object RowEncoder { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | BinaryType => inputObject + case udt: UserDefinedType[_] => + Invoke(inputObject, "serialize", udt.sqlType, inputObject :: Nil) + case TimestampType => StaticInvoke( DateTimeUtils, @@ -129,6 +132,7 @@ object RowEncoder { case _: ArrayType => ObjectType(classOf[scala.collection.Seq[_]]) case _: MapType => ObjectType(classOf[scala.collection.Map[_, _]]) case _: StructType => ObjectType(classOf[Row]) + case udt: UserDefinedType[_] => ObjectType(udt.userClass) } private def constructorFor(schema: StructType): Expression = { @@ -147,6 +151,14 @@ object RowEncoder { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | BinaryType => input + case udt: UserDefinedType[_] => + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + false, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) + Invoke(obj, "deserialize", ObjectType(udt.userClass), input :: Nil) + case TimestampType => StaticInvoke( DateTimeUtils, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 18cc2a63ec19..69bda28c9775 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -43,7 +43,7 @@ case class ScalaUDF( def userDefinedFunc(): AnyRef = function def getChildren(): Seq[Expression] = children - val inputSchema: StructType = { + lazy val inputSchema: StructType = { val fields = if (inputTypes == Nil) { // from the deprecated callUDF codepath children.zipWithIndex.map { case (e, i) => @@ -372,8 +372,9 @@ case class ScalaUDF( } override def eval(input: InternalRow): Any = { + val projected = InternalRow.fromSeq(children.map(_.eval(input))) val outputEncoder: ExpressionEncoder[Row] = RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - outputEncoder.toRow(Row(f(input))) + outputEncoder.toRow(Row(f(projected))).asInstanceOf[InternalRow].get(0, dataType) } } From 75ffaebc56c2bcc692c8924cd31c4b18a61f02c7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Nov 2015 02:37:00 +0800 Subject: [PATCH 03/25] Fix scala style. --- .../org/apache/spark/sql/catalyst/encoders/RowEncoder.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index a8af11b5b45f..3943f9bff2ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -50,7 +50,7 @@ object RowEncoder { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | BinaryType => inputObject - case udt: UserDefinedType[_] => + case udt: UserDefinedType[_] => Invoke(inputObject, "serialize", udt.sqlType, inputObject :: Nil) case TimestampType => @@ -151,7 +151,7 @@ object RowEncoder { case BooleanType | ByteType | ShortType | IntegerType | LongType | FloatType | DoubleType | BinaryType => input - case udt: UserDefinedType[_] => + case udt: UserDefinedType[_] => val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, From 1e13ff98d1458b78c5cb8cb187bad4d91dc1143e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Nov 2015 07:12:16 +0800 Subject: [PATCH 04/25] Call serialize on udt instead of user class. --- .../apache/spark/sql/catalyst/encoders/RowEncoder.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 3943f9bff2ec..a760dee27639 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -51,7 +51,12 @@ object RowEncoder { FloatType | DoubleType | BinaryType => inputObject case udt: UserDefinedType[_] => - Invoke(inputObject, "serialize", udt.sqlType, inputObject :: Nil) + val obj = NewInstance( + udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), + Nil, + false, + dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) + Invoke(obj, "serialize", udt.sqlType, inputObject :: Nil) case TimestampType => StaticInvoke( From 07ff97ad563b63428dc4395f50471df7607818ca Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Nov 2015 10:30:01 +0800 Subject: [PATCH 05/25] Add getField for UserDefinedType. --- .../org/apache/spark/sql/catalyst/encoders/RowEncoder.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index a760dee27639..09ee9f03c09c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -251,5 +251,7 @@ object RowEncoder { Invoke(row, "getArray", dataType, Literal(ordinal) :: Nil) case _: MapType => Invoke(row, "getMap", dataType, Literal(ordinal) :: Nil) + case udt: UserDefinedType[_] => + getField(row, ordinal, udt.sqlType) } } From ecf01bfb070644337681631c08408e0610a433e8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 10 Nov 2015 18:53:26 +0800 Subject: [PATCH 06/25] Move outputEncoder outside of eval and add calling copy(). --- .../apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 69bda28c9775..de9b2683524c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -371,10 +371,11 @@ case class ScalaUDF( """ } + lazy val outputEncoder: ExpressionEncoder[Row] = + RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) + override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) - val outputEncoder: ExpressionEncoder[Row] = - RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - outputEncoder.toRow(Row(f(projected))).asInstanceOf[InternalRow].get(0, dataType) + outputEncoder.toRow(Row(f(projected))).copy().asInstanceOf[InternalRow].get(0, dataType) } } From 39c0b7ad805b04235755b0d339635dae0e7ffbec Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 11 Nov 2015 15:52:30 +0800 Subject: [PATCH 07/25] Replace catalyst converter with RowEncoder for the generated ScalaUDF. --- .../main/scala/org/apache/spark/sql/Row.scala | 10 ++- .../sql/catalyst/encoders/RowEncoder.scala | 7 +- .../sql/catalyst/expressions/ScalaUDF.scala | 82 +++++++++++-------- .../scala/org/apache/spark/sql/UDFSuite.scala | 67 +++++++++------ 4 files changed, 103 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index ed2fdf9f2f7c..d9e5ef33df55 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -306,7 +306,15 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ - def getStruct(i: Int): Row = getAs[Row](i) + def getStruct(i: Int): Row = { + // Product and Row both are recoginized as StructType in a Row + val t = get(i) + if (t.isInstanceOf[Product]) { + Row.fromTuple(t.asInstanceOf[Product]) + } else { + t.asInstanceOf[Row] + } + } /** * Returns the value at position i. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 09ee9f03c09c..e1e487c71f1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -117,11 +117,16 @@ object RowEncoder { case StructType(fields) => val convertedFields = fields.zipWithIndex.map { case (f, i) => + val method = if (f.dataType.isInstanceOf[StructType]) { + "getStruct" + } else { + "get" + } If( Invoke(inputObject, "isNullAt", BooleanType, Literal(i) :: Nil), Literal.create(null, f.dataType), extractorsFor( - Invoke(inputObject, "get", externalDataTypeFor(f.dataType), Literal(i) :: Nil), + Invoke(inputObject, method, externalDataTypeFor(f.dataType), Literal(i) :: Nil), f.dataType)) } CreateStruct(convertedFields) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index de9b2683524c..096bcce14984 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.JavaConversions + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} @@ -42,6 +44,8 @@ case class ScalaUDF( // Accessors used in genCode def userDefinedFunc(): AnyRef = function def getChildren(): Seq[Expression] = children + def getDataType(): StructType = StructType(StructField("_c0", dataType) :: Nil) + def getInputSchema(): StructType = inputSchema lazy val inputSchema: StructType = { val fields = if (inputTypes == Nil) { @@ -294,24 +298,6 @@ case class ScalaUDF( // scalastyle:on - // Generate codes used to convert the arguments to Scala type for user-defined funtions - private[this] def genCodeForConverter( - ctx: CodeGenContext, - scalaUDFTermIdx: Int, - index: Int): String = { - val converterClassName = classOf[Any => Any].getName - val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" - val expressionClassName = classOf[Expression].getName - val scalaUDFClassName = classOf[ScalaUDF].getName - - val converterTerm = ctx.freshName("converter") - ctx.addMutableState(converterClassName, converterTerm, - s"this.$converterTerm = ($converterClassName)$typeConvertersClassName" + - s".createToScalaConverter(((${expressionClassName})((($scalaUDFClassName)" + - s"expressions[$scalaUDFTermIdx]).getChildren().apply($index))).dataType());") - converterTerm - } - override def genCode( ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -323,20 +309,29 @@ case class ScalaUDF( val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" val expressionClassName = classOf[Expression].getName - - // Generate codes used to convert the returned value of user-defined functions to Catalyst type - val catalystConverterTerm = ctx.freshName("catalystConverter") - ctx.addMutableState(converterClassName, catalystConverterTerm, - s"this.$catalystConverterTerm = ($converterClassName)$typeConvertersClassName" + - s".createToCatalystConverter((($scalaUDFClassName)expressions" + - s"[$scalaUDFTermIdx]).dataType());") + val expressionEncoderClassName = classOf[ExpressionEncoder[Row]].getName + val rowEncoderClassName = RowEncoder.getClass.getName + ".MODULE$" + val structTypeClassName = StructType.getClass.getName + ".MODULE$" + val rowClassName = Row.getClass.getName + ".MODULE$" + val rowClass = classOf[Row].getName + val internalRowClassName = classOf[InternalRow].getName + val javaConversionClassName = JavaConversions.getClass.getName + ".MODULE$" + + // Generate code for input encoder + val inputExpressionEncoderTerm = ctx.freshName("inputExpressionEncoder") + ctx.addMutableState(expressionEncoderClassName, inputExpressionEncoderTerm, + s"this.$inputExpressionEncoderTerm = ($expressionEncoderClassName)$rowEncoderClassName" + + s".apply((($scalaUDFClassName)expressions" + + s"[$scalaUDFTermIdx]).getInputSchema());") + + // Generate code for output encoder + val outputExpressionEncoderTerm = ctx.freshName("outputExpressionEncoder") + ctx.addMutableState(expressionEncoderClassName, outputExpressionEncoderTerm, + s"this.$outputExpressionEncoderTerm = ($expressionEncoderClassName)$rowEncoderClassName" + + s".apply((($scalaUDFClassName)expressions[$scalaUDFTermIdx]).getDataType());") val resultTerm = ctx.freshName("result") - // This must be called before children expressions' codegen - // because ctx.references is used in genCodeForConverter - val converterTerms = (0 until children.size).map(genCodeForConverter(ctx, scalaUDFTermIdx, _)) - // Initialize user-defined function val funcClassName = s"scala.Function${children.size}" @@ -347,23 +342,42 @@ case class ScalaUDF( // codegen for children expressions val evals = children.map(_.gen(ctx)) + val evalsArgs = evals.map(_.value).mkString(", ") + val evalsAsSeq = s"$javaConversionClassName.asScalaIterable" + + s"(java.util.Arrays.asList($evalsArgs)).toList()" + val inputInternalRowTerm = ctx.freshName("inputRow") + val inputInternalRow = s"$rowClass $inputInternalRowTerm = " + + s"($rowClass)$inputExpressionEncoderTerm.fromRow(InternalRow.fromSeq($evalsAsSeq));" // Generate the codes for expressions and calling user-defined function // We need to get the boxedType of dataType's javaType here. Because for the dataType // such as IntegerType, its javaType is `int` and the returned type of user-defined // function is Object. Trying to convert an Object to `int` will cause casting exception. val evalCode = evals.map(_.code).mkString - val funcArguments = converterTerms.zip(evals).map { - case (converter, eval) => s"$converter.apply(${eval.value})" - }.mkString(",") + + val funcArguments = (0 until children.size).map { i => + s"$inputInternalRowTerm.get($i)" + }.mkString(", ") + + val rowParametersTerm = ctx.freshName("rowParameters") + val innerRow = s"$rowClass $rowParametersTerm = $rowClassName.apply(" + + s"$javaConversionClassName.asScalaIterable" + + s"(java.util.Arrays.asList($funcTerm.apply($funcArguments))).toList());" + val internalRowTerm = ctx.freshName("internalRow") + val internalRow = s"$internalRowClassName $internalRowTerm = ($internalRowClassName)" + + s"${outputExpressionEncoderTerm}.toRow($rowParametersTerm).copy();" + + val udfDataType = s"(($scalaUDFClassName)expressions[$scalaUDFTermIdx]).dataType()" val callFunc = s"${ctx.boxedType(ctx.javaType(dataType))} $resultTerm = " + - s"(${ctx.boxedType(ctx.javaType(dataType))})${catalystConverterTerm}" + - s".apply($funcTerm.apply($funcArguments));" + s"(${ctx.boxedType(ctx.javaType(dataType))}) $internalRowTerm.get(0, $udfDataType);" evalCode + s""" ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; Boolean ${ev.isNull}; + $inputInternalRow + $innerRow + $internalRow $callFunc ${ev.value} = $resultTerm; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 9837fa6bdb35..973834f32687 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -204,32 +204,45 @@ class UDFSuite extends QueryTest with SharedSQLContext { sqlContext.udf.register("complexDataFunc", (m: Map[String, Int], a: Seq[Int], b: Boolean) => { (m, a, b) } ) - checkAnswer( - sql("SELECT tmp.t.* FROM (SELECT testDataFunc(key, value) AS t from testData) tmp").toDF(), - testData) - checkAnswer( - sql(""" - | SELECT tmp.t.* FROM - | (SELECT decimalDataFunc(a, b) AS t FROM decimalData) tmp - """.stripMargin).toDF(), decimalData) - checkAnswer( - sql(""" - | SELECT tmp.t.* FROM - | (SELECT binaryDataFunc(a, b) AS t FROM binaryData) tmp - """.stripMargin).toDF(), binaryData) - checkAnswer( - sql(""" - | SELECT tmp.t.* FROM - | (SELECT arrayDataFunc(data, nestedData) AS t FROM arrayData) tmp - """.stripMargin).toDF(), arrayData.toDF()) - checkAnswer( - sql(""" - | SELECT mapDataFunc(data) AS t FROM mapData - """.stripMargin).toDF(), mapData.toDF()) - checkAnswer( - sql(""" - | SELECT tmp.t.* FROM - | (SELECT complexDataFunc(m, a, b) AS t FROM complexData) tmp - """.stripMargin).toDF(), complexData.select("m", "a", "b")) + def udfTest(): Unit = { + checkAnswer( + sql("SELECT tmp.t.* FROM (SELECT testDataFunc(key, value) AS t from testData) tmp").toDF(), + testData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT decimalDataFunc(a, b) AS t FROM decimalData) tmp + """.stripMargin).toDF(), decimalData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT binaryDataFunc(a, b) AS t FROM binaryData) tmp + """.stripMargin).toDF(), binaryData) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT arrayDataFunc(data, nestedData) AS t FROM arrayData) tmp + """.stripMargin).toDF(), arrayData.toDF()) + checkAnswer( + sql(""" + | SELECT mapDataFunc(data) AS t FROM mapData + """.stripMargin).toDF(), mapData.toDF()) + checkAnswer( + sql(""" + | SELECT tmp.t.* FROM + | (SELECT complexDataFunc(m, a, b) AS t FROM complexData) tmp + """.stripMargin).toDF(), complexData.select("m", "a", "b")) + } + + withSQLConf(SQLConf.UNSAFE_ENABLED.key -> "true") { + withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "true") { + udfTest() + } + } + withSQLConf(SQLConf.UNSAFE_ENABLED.key -> "false") { + withSQLConf(SQLConf.CODEGEN_ENABLED.key -> "false") { + udfTest() + } + } } } From c910e6edf9e0d9b7307c981413602706be2f14de Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 11 Nov 2015 16:27:59 +0800 Subject: [PATCH 08/25] Fix scala style. --- .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 096bcce14984..c53f5e876472 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import scala.collection.JavaConversions - import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} @@ -315,7 +313,7 @@ case class ScalaUDF( val rowClassName = Row.getClass.getName + ".MODULE$" val rowClass = classOf[Row].getName val internalRowClassName = classOf[InternalRow].getName - val javaConversionClassName = JavaConversions.getClass.getName + ".MODULE$" + val javaConversionClassName = scala.collection.JavaConversions.getClass.getName + ".MODULE$" // Generate code for input encoder val inputExpressionEncoderTerm = ctx.freshName("inputExpressionEncoder") From 12345150cff5c02780e0055600b584a0aeaaf441 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 11 Nov 2015 17:00:54 +0800 Subject: [PATCH 09/25] Fix scala style. --- .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index c53f5e876472..3c077ba5caf6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -313,7 +313,9 @@ case class ScalaUDF( val rowClassName = Row.getClass.getName + ".MODULE$" val rowClass = classOf[Row].getName val internalRowClassName = classOf[InternalRow].getName + // scalastyle:off val javaConversionClassName = scala.collection.JavaConversions.getClass.getName + ".MODULE$" + // scalastyle:on // Generate code for input encoder val inputExpressionEncoderTerm = ctx.freshName("inputExpressionEncoder") From 2c8571444eadb962529e076a54655012fd44e242 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 7 Dec 2015 15:36:12 +0800 Subject: [PATCH 10/25] Use reflection to call function for interpreted version. Add more comments. --- .../sql/catalyst/expressions/ScalaUDF.scala | 328 +++++------------- .../scala/org/apache/spark/sql/UDFSuite.scala | 13 + 2 files changed, 97 insertions(+), 244 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 5f42c584fc05..7f850c010a6d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.reflect.runtime.universe._ + import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} @@ -45,11 +47,9 @@ case class ScalaUDF( override def toString: String = s"UDF(${children.mkString(",")})" - // Accessors used in genCode - def userDefinedFunc(): AnyRef = function - def getChildren(): Seq[Expression] = children + // The dataType used in output expression encoder + // The return values of UDF will be encoded in a field in an internal row def getDataType(): StructType = StructType(StructField("_c0", dataType) :: Nil) - def getInputSchema(): StructType = inputSchema lazy val inputSchema: StructType = { val fields = if (inputTypes == Nil) { @@ -65,243 +65,6 @@ case class ScalaUDF( StructType(fields) } - // scalastyle:off - - /** This method has been generated by this script - - (1 to 22).map { x => - val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val evals = (0 to x - 1).map(x => s"convertedRow.get($x)").reduce(_ + ",\n " + _) - - s"""case $x => - val func = function.asInstanceOf[($anys) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func( - $evals) - } - """ - }.foreach(println) - - */ - - private[this] val f = { - lazy val inputEncoder: ExpressionEncoder[Row] = RowEncoder(inputSchema) - children.size match { - case 0 => - val func = function.asInstanceOf[() => Any] - (input: InternalRow) => { - func() - } - - case 1 => - val func = function.asInstanceOf[(Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0)) - } - - case 2 => - val func = function.asInstanceOf[(Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1)) - } - - case 3 => - val func = function.asInstanceOf[(Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2)) - } - - case 4 => - val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3)) - } - - case 5 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4)) - } - - case 6 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5)) - } - - case 7 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6)) - } - - case 8 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7)) - } - - case 9 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8)) - } - - case 10 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9)) - } - - case 11 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10)) - } - - case 12 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11)) - } - - case 13 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12)) - } - - case 14 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13)) - } - - case 15 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14)) - } - - case 16 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15)) - } - - case 17 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16)) - } - - case 18 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16), convertedRow.get(17)) - } - - case 19 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16), convertedRow.get(17), convertedRow.get(18)) - } - - case 20 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19)) - } - - case 21 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19), - convertedRow.get(20)) - } - - case 22 => - val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] - (input: InternalRow) => { - val convertedRow: Row = inputEncoder.fromRow(input) - func(convertedRow.get(0), convertedRow.get(1), convertedRow.get(2), convertedRow.get(3), - convertedRow.get(4), convertedRow.get(5), convertedRow.get(6), convertedRow.get(7), - convertedRow.get(8), convertedRow.get(9), convertedRow.get(10), convertedRow.get(11), - convertedRow.get(12), convertedRow.get(13), convertedRow.get(14), convertedRow.get(15), - convertedRow.get(16), convertedRow.get(17), convertedRow.get(18), convertedRow.get(19), - convertedRow.get(20), convertedRow.get(21)) - } - } - } - - // scalastyle:on - override def genCode( ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { @@ -320,6 +83,10 @@ case class ScalaUDF( val rowClass = classOf[Row].getName val internalRowClassName = classOf[InternalRow].getName // scalastyle:off + // JavaConversions has been banned for implicit conversion between Java and Scala types. + // However, we are not going to use it in Scala side but use it in generated Java codes. + // JavaConverters doesn't provide simple and direct method to call for the purpose here. + // So we turn off scalastyle here temporarily. val javaConversionClassName = scala.collection.JavaConversions.getClass.getName + ".MODULE$" // scalastyle:on @@ -328,7 +95,7 @@ case class ScalaUDF( ctx.addMutableState(expressionEncoderClassName, inputExpressionEncoderTerm, s"this.$inputExpressionEncoderTerm = ($expressionEncoderClassName)$rowEncoderClassName" + s".apply((($scalaUDFClassName)expressions" + - s"[$scalaUDFTermIdx]).getInputSchema());") + s"[$scalaUDFTermIdx]).inputSchema());") // Generate code for output encoder val outputExpressionEncoderTerm = ctx.freshName("outputExpressionEncoder") @@ -344,13 +111,15 @@ case class ScalaUDF( val funcTerm = ctx.freshName("udf") ctx.addMutableState(funcClassName, funcTerm, s"this.$funcTerm = ($funcClassName)((($scalaUDFClassName)expressions" + - s"[$scalaUDFTermIdx]).userDefinedFunc());") + s"[$scalaUDFTermIdx]).function());") // codegen for children expressions val evals = children.map(_.gen(ctx)) val evalsArgs = evals.map(_.value).mkString(", ") val evalsAsSeq = s"$javaConversionClassName.asScalaIterable" + s"(java.util.Arrays.asList($evalsArgs)).toList()" + + // Encode children expression results to Scala objects val inputInternalRowTerm = ctx.freshName("inputRow") val inputInternalRow = s"$rowClass $inputInternalRowTerm = " + s"($rowClass)$inputExpressionEncoderTerm.fromRow(InternalRow.fromSeq($evalsAsSeq));" @@ -369,10 +138,14 @@ case class ScalaUDF( val innerRow = s"$rowClass $rowParametersTerm = $rowClassName.apply(" + s"$javaConversionClassName.asScalaIterable" + s"(java.util.Arrays.asList($funcTerm.apply($funcArguments))).toList());" + + // Encode Scala objects of UDF return values to Spark SQL internal row val internalRowTerm = ctx.freshName("internalRow") val internalRow = s"$internalRowClassName $internalRowTerm = ($internalRowClassName)" + s"${outputExpressionEncoderTerm}.toRow($rowParametersTerm).copy();" + // UDF return values are encoded as the field 0 as StructType in the internal row + // We extract it back val udfDataType = s"(($scalaUDFClassName)expressions[$scalaUDFTermIdx]).dataType()" val callFunc = s"${ctx.boxedType(ctx.javaType(dataType))} $resultTerm = " + s"(${ctx.boxedType(ctx.javaType(dataType))}) $internalRowTerm.get(0, $udfDataType);" @@ -391,11 +164,78 @@ case class ScalaUDF( """ } + lazy val inputEncoder: ExpressionEncoder[Row] = RowEncoder(inputSchema) lazy val outputEncoder: ExpressionEncoder[Row] = RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) override def eval(input: InternalRow): Any = { + val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) + val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) + .asTerm.alternatives + val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) + val projected = InternalRow.fromSeq(children.map(_.eval(input))) - outputEncoder.toRow(Row(f(projected))).copy().asInstanceOf[InternalRow].get(0, dataType) + val cRow: Row = inputEncoder.fromRow(projected) + + val callRet = children.size match { + case 0 => invokeMethod() + case 1 => invokeMethod(cRow(0)) + case 2 => invokeMethod(cRow(0), cRow(1)) + case 3 => invokeMethod(cRow(0), cRow(1), cRow(2)) + case 4 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3)) + case 5 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4)) + case 6 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5)) + case 7 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6)) + case 8 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7)) + case 9 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8)) + case 10 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9)) + case 11 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10)) + case 12 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11)) + case 13 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12)) + case 14 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13)) + case 15 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14)) + case 16 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15)) + case 17 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16)) + case 18 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17)) + case 19 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18)) + case 20 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19)) + case 21 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19), cRow(20)) + case 22 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19), cRow(20), cRow(21)) + } + + outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 240decb62c49..083a28b5cada 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ @@ -192,6 +193,18 @@ class UDFSuite extends QueryTest with SharedSQLContext { assert(sql("SELECT intExpected(1.0)").head().getInt(0) === 1) } + test("udf in interpreted projection") { + sqlContext.udf.register("testDataFunc", (n: Int, s: String) => { (n, s) }) + + // TakeOrderedAndProject uses InterpretedProjection to do projection + // So this SQL will call interpreted version of ScalaUDF + checkAnswer( + sql(""" + | SELECT testDataFunc(t.key, t.value) FROM + | (SELECT key, value FROM testData ORDER BY key) t LIMIT 100 + """.stripMargin).toDF(), testData.select(struct("key", "value"))) + } + test("udf in different types") { sqlContext.udf.register("testDataFunc", (n: Int, s: String) => { (n, s) }) sqlContext.udf.register("testDataFunc2", (ns: Row) => { (ns.getInt(0), ns.getString(1)) }) From f80675504a34d5bc0b830b5d25eb2f7f9a637d74 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 7 Dec 2015 15:41:47 +0800 Subject: [PATCH 11/25] Move reflection code outside eval function. --- .../apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 7f850c010a6d..e7f7e7cbb2ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -168,12 +168,12 @@ case class ScalaUDF( lazy val outputEncoder: ExpressionEncoder[Row] = RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - override def eval(input: InternalRow): Any = { - val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) - val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) + lazy val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) + lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) .asTerm.alternatives - val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) + lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) + override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) val cRow: Row = inputEncoder.fromRow(projected) From 26b4d8583980f671513c7d4d532260bce5b86667 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 7 Dec 2015 22:27:16 +0800 Subject: [PATCH 12/25] Process exception thrown in UDF. --- .../sql/catalyst/expressions/ScalaUDF.scala | 124 ++++++++++-------- 1 file changed, 67 insertions(+), 57 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index e7f7e7cbb2ff..be1738a66b49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.lang.reflect.InvocationTargetException + import scala.reflect.runtime.universe._ import org.apache.spark.sql.catalyst.InternalRow @@ -177,63 +179,71 @@ case class ScalaUDF( val projected = InternalRow.fromSeq(children.map(_.eval(input))) val cRow: Row = inputEncoder.fromRow(projected) - val callRet = children.size match { - case 0 => invokeMethod() - case 1 => invokeMethod(cRow(0)) - case 2 => invokeMethod(cRow(0), cRow(1)) - case 3 => invokeMethod(cRow(0), cRow(1), cRow(2)) - case 4 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3)) - case 5 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4)) - case 6 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5)) - case 7 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6)) - case 8 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7)) - case 9 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8)) - case 10 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9)) - case 11 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10)) - case 12 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11)) - case 13 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12)) - case 14 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13)) - case 15 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14)) - case 16 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15)) - case 17 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16)) - case 18 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17)) - case 19 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18)) - case 20 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19)) - case 21 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19), cRow(20)) - case 22 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19), cRow(20), cRow(21)) + var callRet: Any = null + try { + callRet = children.size match { + case 0 => invokeMethod() + case 1 => invokeMethod(cRow(0)) + case 2 => invokeMethod(cRow(0), cRow(1)) + case 3 => invokeMethod(cRow(0), cRow(1), cRow(2)) + case 4 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3)) + case 5 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4)) + case 6 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5)) + case 7 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6)) + case 8 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7)) + case 9 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8)) + case 10 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9)) + case 11 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10)) + case 12 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11)) + case 13 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12)) + case 14 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13)) + case 15 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14)) + case 16 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15)) + case 17 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16)) + case 18 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17)) + case 19 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18)) + case 20 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19)) + case 21 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19), cRow(20)) + case 22 => + invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), + cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), + cRow(17), cRow(18), cRow(19), cRow(20), cRow(21)) + } + } catch { + // When exception is thrown in UDF, an InvocationTargetException will be thrown. + // We get and re-throw the cause exception. + case e: InvocationTargetException => throw e.getTargetException } outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) From 693a6fed9fdb8002d4e099bd106ec1b44dbdc86d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 8 Dec 2015 14:48:50 +0800 Subject: [PATCH 13/25] Try to solve failed tests. --- .../sql/catalyst/expressions/ScalaUDF.scala | 68 ++----------------- 1 file changed, 4 insertions(+), 64 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index be1738a66b49..f89f0bc9fd89 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -172,80 +172,20 @@ case class ScalaUDF( lazy val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) - .asTerm.alternatives - lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) + .asTerm.alternatives + lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _ override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) val cRow: Row = inputEncoder.fromRow(projected) - var callRet: Any = null try { - callRet = children.size match { - case 0 => invokeMethod() - case 1 => invokeMethod(cRow(0)) - case 2 => invokeMethod(cRow(0), cRow(1)) - case 3 => invokeMethod(cRow(0), cRow(1), cRow(2)) - case 4 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3)) - case 5 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4)) - case 6 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5)) - case 7 => invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6)) - case 8 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7)) - case 9 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8)) - case 10 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9)) - case 11 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10)) - case 12 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11)) - case 13 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12)) - case 14 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13)) - case 15 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14)) - case 16 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15)) - case 17 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16)) - case 18 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17)) - case 19 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18)) - case 20 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19)) - case 21 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19), cRow(20)) - case 22 => - invokeMethod(cRow(0), cRow(1), cRow(2), cRow(3), cRow(4), cRow(5), cRow(6), cRow(7), - cRow(8), cRow(9), cRow(10), cRow(11), cRow(12), cRow(13), cRow(14), cRow(15), cRow(16), - cRow(17), cRow(18), cRow(19), cRow(20), cRow(21)) - } + val callRet = invokeMethod(cRow.toSeq) + outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) } catch { // When exception is thrown in UDF, an InvocationTargetException will be thrown. // We get and re-throw the cause exception. case e: InvocationTargetException => throw e.getTargetException } - - outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) } } From 1ca2efcaa2eb511a398b5db67dee5f340a524de2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 8 Dec 2015 18:16:58 +0800 Subject: [PATCH 14/25] Try again. --- .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index f89f0bc9fd89..98ed1529329e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -173,7 +173,8 @@ case class ScalaUDF( lazy val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) .asTerm.alternatives - lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _ + lazy val invokeMethod = (reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _) + .asInstanceOf[Seq[Any] => Any] override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) From 2fcbe69b7b626201736a16867f3c1feefc834ccf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 1 Apr 2016 14:26:15 +0000 Subject: [PATCH 15/25] Fix scala style. --- .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index c03283733f2f..19b1ab9e80d8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -21,7 +21,6 @@ import java.lang.reflect.InvocationTargetException import scala.reflect.runtime.universe._ -import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -76,7 +75,7 @@ case class ScalaUDF( val scalaUDFClassName = classOf[ScalaUDF].getName val scalaUDFObject = ctx.addReferenceObj("scalaUDF", this, scalaUDFClassName) - + val converterClassName = classOf[Any => Any].getName val typeConvertersClassName = CatalystTypeConverters.getClass.getName + ".MODULE$" val expressionClassName = classOf[Expression].getName From 5da1c1390e6da81321f2be76e20c92ee03c05ebf Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 2 Apr 2016 04:34:53 +0000 Subject: [PATCH 16/25] Try it. --- .../sql/catalyst/expressions/ScalaUDF.scala | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 19b1ab9e80d8..04923ea8aeeb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -169,15 +169,31 @@ case class ScalaUDF( lazy val outputEncoder: ExpressionEncoder[Row] = RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - lazy val reflectedFunc = runtimeMirror(function.getClass.getClassLoader).reflect(function) + // scalastyle:off + lazy val convertedFunc = children.size match { + case 0 => function.asInstanceOf[Function0[Any]] + case 1 => function.asInstanceOf[Function1[Any, Any]] + case 2 => function.asInstanceOf[Function2[Any, Any, Any]] + case 3 => function.asInstanceOf[Function3[Any, Any, Any, Any]] + case 4 => function.asInstanceOf[Function4[Any, Any, Any, Any, Any]] + case 5 => function.asInstanceOf[Function5[Any, Any, Any, Any, Any, Any]] + case 6 => function.asInstanceOf[Function6[Any, Any, Any, Any, Any, Any, Any]] + case 7 => function.asInstanceOf[Function7[Any, Any, Any, Any, Any, Any, Any, Any]] + case 8 => function.asInstanceOf[Function8[Any, Any, Any, Any, Any, Any, Any, Any, Any]] + case 9 => function.asInstanceOf[Function9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + case 10 => function.asInstanceOf[Function10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] + } + // scalastyle:on + + lazy val reflectedFunc = + runtimeMirror(convertedFunc.getClass.getClassLoader).reflect(convertedFunc) lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) .asTerm.alternatives - lazy val invokeMethod = (reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _) - .asInstanceOf[Seq[Any] => Any] + lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _ override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) - val cRow: Row = inputEncoder.fromRow(projected) + val cRow = inputEncoder.fromRow(projected) try { val callRet = invokeMethod(cRow.toSeq) From 60f4ca00792dcbddb55c25d7e18bd61aeefacd28 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 04:52:08 +0000 Subject: [PATCH 17/25] Try to fix test. --- .../sql/catalyst/expressions/ScalaUDF.scala | 22 +++---------------- 1 file changed, 3 insertions(+), 19 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 04923ea8aeeb..d561fcc9dd62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -169,34 +169,18 @@ case class ScalaUDF( lazy val outputEncoder: ExpressionEncoder[Row] = RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - // scalastyle:off - lazy val convertedFunc = children.size match { - case 0 => function.asInstanceOf[Function0[Any]] - case 1 => function.asInstanceOf[Function1[Any, Any]] - case 2 => function.asInstanceOf[Function2[Any, Any, Any]] - case 3 => function.asInstanceOf[Function3[Any, Any, Any, Any]] - case 4 => function.asInstanceOf[Function4[Any, Any, Any, Any, Any]] - case 5 => function.asInstanceOf[Function5[Any, Any, Any, Any, Any, Any]] - case 6 => function.asInstanceOf[Function6[Any, Any, Any, Any, Any, Any, Any]] - case 7 => function.asInstanceOf[Function7[Any, Any, Any, Any, Any, Any, Any, Any]] - case 8 => function.asInstanceOf[Function8[Any, Any, Any, Any, Any, Any, Any, Any, Any]] - case 9 => function.asInstanceOf[Function9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] - case 10 => function.asInstanceOf[Function10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]] - } - // scalastyle:on - lazy val reflectedFunc = - runtimeMirror(convertedFunc.getClass.getClassLoader).reflect(convertedFunc) + runtimeMirror(function.getClass.getClassLoader).reflect(function) lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) .asTerm.alternatives - lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod).apply _ + lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) override def eval(input: InternalRow): Any = { val projected = InternalRow.fromSeq(children.map(_.eval(input))) val cRow = inputEncoder.fromRow(projected) try { - val callRet = invokeMethod(cRow.toSeq) + val callRet = invokeMethod.apply(cRow.toSeq: _*) outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) } catch { // When exception is thrown in UDF, an InvocationTargetException will be thrown. From 7a046fa3864e8e49860205e2ed47e65de99acb4c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 05:16:17 +0000 Subject: [PATCH 18/25] Make createTransformFunc as val. --- mllib/src/main/scala/org/apache/spark/ml/Transformer.scala | 2 +- mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala | 2 +- .../org/apache/spark/ml/feature/ElementwiseProduct.scala | 2 +- mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala | 2 +- .../main/scala/org/apache/spark/ml/feature/Normalizer.scala | 2 +- .../org/apache/spark/ml/feature/PolynomialExpansion.scala | 2 +- .../main/scala/org/apache/spark/ml/feature/Tokenizer.scala | 4 ++-- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 2538c0f477fc..6f4bc36a293f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -90,7 +90,7 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] * account of the embedded param map. So the param values should be determined solely by the input * param map. */ - protected def createTransformFunc: IN => OUT + protected val createTransformFunc: IN => OUT /** * Returns the data type of the output column. diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index a6f878151de7..85aae9311ee9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -57,7 +57,7 @@ class DCT(override val uid: String) setDefault(inverse -> false) - override protected def createTransformFunc: Vector => Vector = { vec => + override protected val createTransformFunc: Vector => Vector = { vec => val result = vec.toArray val jTransformer = new DoubleDCT_1D(result.length) if ($(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 2c7ffdb7ba69..55f3b5c0d720 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -49,7 +49,7 @@ class ElementwiseProduct(override val uid: String) /** @group getParam */ def getScalingVec: Vector = getOrDefault(scalingVec) - override protected def createTransformFunc: Vector => Vector = { + override protected val createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) elemScaler.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index f8bc7e3f0c03..b99bbe7f831b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -56,7 +56,7 @@ class NGram(override val uid: String) setDefault(n -> 2) - override protected def createTransformFunc: Seq[String] => Seq[String] = { + override protected val createTransformFunc: Seq[String] => Seq[String] = { _.iterator.sliding($(n)).withPartial(false).map(_.mkString(" ")).toSeq } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index a603b3f83320..accb209bd578 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -50,7 +50,7 @@ class Normalizer(override val uid: String) /** @group setParam */ def setP(value: Double): this.type = set(p, value) - override protected def createTransformFunc: Vector => Vector = { + override protected val createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) normalizer.transform } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index 0a9b9719c15d..f74732b50d98 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -56,7 +56,7 @@ class PolynomialExpansion(override val uid: String) /** @group setParam */ def setDegree(value: Int): this.type = set(degree, value) - override protected def createTransformFunc: Vector => Vector = { v => + override protected val createTransformFunc: Vector => Vector = { v => PolynomialExpansion.expand(v, $(degree)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 8456a0e91580..2cca2de656ab 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -35,7 +35,7 @@ class Tokenizer(override val uid: String) def this() = this(Identifiable.randomUID("tok")) - override protected def createTransformFunc: String => Seq[String] = { + override protected val createTransformFunc: String => Seq[String] = { _.toLowerCase.split("\\s") } @@ -124,7 +124,7 @@ class RegexTokenizer(override val uid: String) setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) - override protected def createTransformFunc: String => Seq[String] = { originStr => + override protected val createTransformFunc: String => Seq[String] = { originStr => val re = $(pattern).r val str = if ($(toLowercase)) originStr.toLowerCase() else originStr val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq From dd43918e06310eb519474e48e2341e38cfd0bba7 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 07:04:02 +0000 Subject: [PATCH 19/25] Make createTransformFunc as lazy val. --- mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala | 2 +- .../org/apache/spark/ml/feature/ElementwiseProduct.scala | 2 +- mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala | 2 +- .../main/scala/org/apache/spark/ml/feature/Normalizer.scala | 2 +- .../org/apache/spark/ml/feature/PolynomialExpansion.scala | 2 +- .../main/scala/org/apache/spark/ml/feature/Tokenizer.scala | 4 ++-- 6 files changed, 7 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 85aae9311ee9..3d9a559a0926 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -57,7 +57,7 @@ class DCT(override val uid: String) setDefault(inverse -> false) - override protected val createTransformFunc: Vector => Vector = { vec => + override protected lazy val createTransformFunc: Vector => Vector = { vec => val result = vec.toArray val jTransformer = new DoubleDCT_1D(result.length) if ($(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 798196914085..588cbc5e2ed9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -49,7 +49,7 @@ class ElementwiseProduct(override val uid: String) /** @group getParam */ def getScalingVec: Vector = getOrDefault(scalingVec) - override protected val createTransformFunc: Vector => Vector = { + override protected lazy val createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") val elemScaler = new feature.ElementwiseProduct($(scalingVec)) elemScaler.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index b99bbe7f831b..3f264bfb7c4b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -56,7 +56,7 @@ class NGram(override val uid: String) setDefault(n -> 2) - override protected val createTransformFunc: Seq[String] => Seq[String] = { + override protected lazy val createTransformFunc: Seq[String] => Seq[String] = { _.iterator.sliding($(n)).withPartial(false).map(_.mkString(" ")).toSeq } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index accb209bd578..d2553d3a60e8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -50,7 +50,7 @@ class Normalizer(override val uid: String) /** @group setParam */ def setP(value: Double): this.type = set(p, value) - override protected val createTransformFunc: Vector => Vector = { + override protected lazy val createTransformFunc: Vector => Vector = { val normalizer = new feature.Normalizer($(p)) normalizer.transform } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index f74732b50d98..cbb5dff02e5b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -56,7 +56,7 @@ class PolynomialExpansion(override val uid: String) /** @group setParam */ def setDegree(value: Int): this.type = set(degree, value) - override protected val createTransformFunc: Vector => Vector = { v => + override protected lazy val createTransformFunc: Vector => Vector = { v => PolynomialExpansion.expand(v, $(degree)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 2cca2de656ab..72a1f74530b2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -35,7 +35,7 @@ class Tokenizer(override val uid: String) def this() = this(Identifiable.randomUID("tok")) - override protected val createTransformFunc: String => Seq[String] = { + override protected lazy val createTransformFunc: String => Seq[String] = { _.toLowerCase.split("\\s") } @@ -124,7 +124,7 @@ class RegexTokenizer(override val uid: String) setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) - override protected val createTransformFunc: String => Seq[String] = { originStr => + override protected lazy val createTransformFunc: String => Seq[String] = { originStr => val re = $(pattern).r val str = if ($(toLowercase)) originStr.toLowerCase() else originStr val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq From 8dbc551f72c2a940f9fed54c0cfce5a208c3fc60 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 09:32:51 +0000 Subject: [PATCH 20/25] Pass Transformer into UDF to get the updated param values. --- .../scala/org/apache/spark/ml/Transformer.scala | 5 +++-- .../scala/org/apache/spark/ml/feature/DCT.scala | 4 ++-- .../spark/ml/feature/ElementwiseProduct.scala | 9 +++++---- .../org/apache/spark/ml/feature/NGram.scala | 5 +++-- .../apache/spark/ml/feature/Normalizer.scala | 7 ++++--- .../spark/ml/feature/PolynomialExpansion.scala | 4 ++-- .../org/apache/spark/ml/feature/Tokenizer.scala | 17 +++++++++-------- 7 files changed, 28 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 6f4bc36a293f..6f7773c76051 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -90,7 +90,7 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] * account of the embedded param map. So the param values should be determined solely by the input * param map. */ - protected val createTransformFunc: IN => OUT + protected val createTransformFunc: (T, IN) => OUT /** * Returns the data type of the output column. @@ -115,7 +115,8 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] override def transform(dataset: DataFrame): DataFrame = { transformSchema(dataset.schema, logging = true) - val transformUDF = udf(this.createTransformFunc, outputDataType) + val func = (x: IN) => createTransformFunc(copy(ParamMap.empty), x) + val transformUDF = udf(func, outputDataType) dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index 3d9a559a0926..f10adc5d672e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -57,10 +57,10 @@ class DCT(override val uid: String) setDefault(inverse -> false) - override protected lazy val createTransformFunc: Vector => Vector = { vec => + override protected val createTransformFunc: (DCT, Vector) => Vector = { (dct, vec) => val result = vec.toArray val jTransformer = new DoubleDCT_1D(result.length) - if ($(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) + if (dct.$(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) Vectors.dense(result) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 588cbc5e2ed9..626a6457f027 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -49,10 +49,11 @@ class ElementwiseProduct(override val uid: String) /** @group getParam */ def getScalingVec: Vector = getOrDefault(scalingVec) - override protected lazy val createTransformFunc: Vector => Vector = { - require(params.contains(scalingVec), s"transformation requires a weight vector") - val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - elemScaler.transform + override protected val createTransformFunc: (ElementwiseProduct, Vector) => Vector = { + (transformer, vec) => + require(transformer.params.contains(scalingVec), s"transformation requires a weight vector") + val elemScaler = new feature.ElementwiseProduct(transformer.$(scalingVec)) + elemScaler.transform(vec) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index 3f264bfb7c4b..d638d8491f1a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -56,8 +56,9 @@ class NGram(override val uid: String) setDefault(n -> 2) - override protected lazy val createTransformFunc: Seq[String] => Seq[String] = { - _.iterator.sliding($(n)).withPartial(false).map(_.mkString(" ")).toSeq + override protected val createTransformFunc: (NGram, Seq[String]) => Seq[String] = { + (ngram, strings) => + strings.iterator.sliding(ngram.$(n)).withPartial(false).map(_.mkString(" ")).toSeq } override protected def validateInputType(inputType: DataType): Unit = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index d2553d3a60e8..a366868586e9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -50,9 +50,10 @@ class Normalizer(override val uid: String) /** @group setParam */ def setP(value: Double): this.type = set(p, value) - override protected lazy val createTransformFunc: Vector => Vector = { - val normalizer = new feature.Normalizer($(p)) - normalizer.transform + override protected val createTransformFunc: (Normalizer, Vector) => Vector = { + (transformer, vec) => + val normalizer = new feature.Normalizer(transformer.$(p)) + normalizer.transform(vec) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index cbb5dff02e5b..ec8f3d8cb4e5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -56,8 +56,8 @@ class PolynomialExpansion(override val uid: String) /** @group setParam */ def setDegree(value: Int): this.type = set(degree, value) - override protected lazy val createTransformFunc: Vector => Vector = { v => - PolynomialExpansion.expand(v, $(degree)) + override protected val createTransformFunc: (PolynomialExpansion, Vector) => Vector = { (p, v) => + PolynomialExpansion.expand(v, p.$(degree)) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 72a1f74530b2..2f6448da49bb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -35,8 +35,8 @@ class Tokenizer(override val uid: String) def this() = this(Identifiable.randomUID("tok")) - override protected lazy val createTransformFunc: String => Seq[String] = { - _.toLowerCase.split("\\s") + override protected val createTransformFunc: (Tokenizer, String) => Seq[String] = { (_, str) => + str.toLowerCase.split("\\s") } override protected def validateInputType(inputType: DataType): Unit = { @@ -124,12 +124,13 @@ class RegexTokenizer(override val uid: String) setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) - override protected lazy val createTransformFunc: String => Seq[String] = { originStr => - val re = $(pattern).r - val str = if ($(toLowercase)) originStr.toLowerCase() else originStr - val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq - val minLength = $(minTokenLength) - tokens.filter(_.length >= minLength) + override protected val createTransformFunc: (RegexTokenizer, String) => Seq[String] = { + (tokenizer, originStr) => + val re = tokenizer.$(pattern).r + val str = if (tokenizer.$(toLowercase)) originStr.toLowerCase() else originStr + val tokens = if (tokenizer.$(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq + val minLength = tokenizer.$(minTokenLength) + tokens.filter(_.length >= minLength) } override protected def validateInputType(inputType: DataType): Unit = { From 597c9713f1262d18b853b5876614a658b3bb61d3 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 10:40:40 +0000 Subject: [PATCH 21/25] Fix MiMa problem. --- project/MimaExcludes.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index d916c49a6a4d..7f9a8058ec8d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -613,6 +613,17 @@ object MimaExcludes { ) ++ Seq( // [SPARK-13430][ML] moved featureCol from LinearRegressionModelSummary to LinearRegressionSummary ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.ml.regression.LinearRegressionSummary.this") + ) ++ Seq( + // [SPARK-11593][SQL] Replace catalyst converter with RowEncoder in ScalaUDF + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.UnaryTransformer.createTransformFunc"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.UnaryTransformer.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.DCT.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.ElementwiseProduct.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Normalizer.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.PolynomialExpansion.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.NGram.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.Tokenizer.createTransformFunc"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.ml.feature.RegexTokenizer.createTransformFunc") ) case v if v.startsWith("1.6") => Seq( From 405e8b048f475bef2893f3be76b43516ab829954 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 15:01:00 +0000 Subject: [PATCH 22/25] Fix passing null into ScalaUDF. --- .../sql/catalyst/expressions/ScalaUDF.scala | 21 ++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index d561fcc9dd62..ccb7a82713bb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -116,9 +116,14 @@ case class ScalaUDF( // codegen for children expressions val evals = children.map(_.gen(ctx)) - val evalsArgs = evals.map(_.value).mkString(", ") + val (converters, evalsArgs) = evals.zipWithIndex.map { case (eval, i) => + val argTerm = ctx.freshName("arg") + val convert = s"${ctx.boxedType(children(i).dataType)} $argTerm = ${eval.isNull} ? null " + + s": new ${ctx.boxedType(children(i).dataType)}(${eval.value});" + (convert, argTerm) + }.unzip val evalsAsSeq = s"$javaConversionClassName.collectionAsScalaIterable" + - s"(java.util.Arrays.asList($evalsArgs)).toList()" + s"(java.util.Arrays.asList(${evalsArgs.mkString(", ")})).toList()" // Encode children expression results to Scala objects val inputInternalRowTerm = ctx.freshName("inputRow") @@ -148,20 +153,22 @@ case class ScalaUDF( // UDF return values are encoded as the field 0 as StructType in the internal row // We extract it back val udfDataType = s"$scalaUDFObject.dataType()" - val callFunc = s"${ctx.boxedType(ctx.javaType(dataType))} $resultTerm = " + - s"(${ctx.boxedType(ctx.javaType(dataType))}) $internalRowTerm.get(0, $udfDataType);" + val callFunc = s"${ctx.boxedType(dataType)} $resultTerm = " + + s"(${ctx.boxedType(dataType)}) $internalRowTerm.get(0, $udfDataType);" evalCode + s""" + ${converters.mkString("\n")} ${ctx.javaType(dataType)} ${ev.value} = ${ctx.defaultValue(dataType)}; - Boolean ${ev.isNull}; $inputInternalRow $innerRow $internalRow $callFunc - ${ev.value} = $resultTerm; - ${ev.isNull} = $resultTerm == null; + boolean ${ev.isNull} = $resultTerm == null; + if (!${ev.isNull}) { + ${ev.value} = $resultTerm; + } """ } From 648c7b22dbea6de9b4a57fab87941bb8cac268bb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 7 Apr 2016 23:21:16 +0000 Subject: [PATCH 23/25] Check PrimitiveType. --- .../apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index ccb7a82713bb..d38a2bb61792 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -118,8 +118,13 @@ case class ScalaUDF( val evals = children.map(_.gen(ctx)) val (converters, evalsArgs) = evals.zipWithIndex.map { case (eval, i) => val argTerm = ctx.freshName("arg") - val convert = s"${ctx.boxedType(children(i).dataType)} $argTerm = ${eval.isNull} ? null " + - s": new ${ctx.boxedType(children(i).dataType)}(${eval.value});" + val convert = if (ctx.isPrimitiveType(children(i).dataType)) { + s"${ctx.boxedType(children(i).dataType)} $argTerm = ${eval.isNull} ? null " + + s": new ${ctx.boxedType(children(i).dataType)}(${eval.value});" + } else { + s"${ctx.boxedType(children(i).dataType)} $argTerm = ${eval.isNull} ? null " + + s": (${ctx.boxedType(children(i).dataType)}) ${eval.value};" + } (convert, argTerm) }.unzip val evalsAsSeq = s"$javaConversionClassName.collectionAsScalaIterable" + From 884a176a0e19ccbaf42f98ad5c1f256dbc10f92b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 11 Apr 2016 02:56:28 +0000 Subject: [PATCH 24/25] Remove ScalaUDF non code-generated evaluation support. --- .../org/apache/spark/ml/Transformer.scala | 5 ++-- .../org/apache/spark/ml/feature/DCT.scala | 4 +-- .../spark/ml/feature/ElementwiseProduct.scala | 9 +++---- .../org/apache/spark/ml/feature/NGram.scala | 5 ++-- .../apache/spark/ml/feature/Normalizer.scala | 7 +++--- .../ml/feature/PolynomialExpansion.scala | 4 +-- .../apache/spark/ml/feature/Tokenizer.scala | 17 ++++++------- .../sql/catalyst/expressions/ScalaUDF.scala | 25 ++----------------- 8 files changed, 25 insertions(+), 51 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 6f7773c76051..2538c0f477fc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -90,7 +90,7 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] * account of the embedded param map. So the param values should be determined solely by the input * param map. */ - protected val createTransformFunc: (T, IN) => OUT + protected def createTransformFunc: IN => OUT /** * Returns the data type of the output column. @@ -115,8 +115,7 @@ abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] override def transform(dataset: DataFrame): DataFrame = { transformSchema(dataset.schema, logging = true) - val func = (x: IN) => createTransformFunc(copy(ParamMap.empty), x) - val transformUDF = udf(func, outputDataType) + val transformUDF = udf(this.createTransformFunc, outputDataType) dataset.withColumn($(outputCol), transformUDF(dataset($(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala index f10adc5d672e..a6f878151de7 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/DCT.scala @@ -57,10 +57,10 @@ class DCT(override val uid: String) setDefault(inverse -> false) - override protected val createTransformFunc: (DCT, Vector) => Vector = { (dct, vec) => + override protected def createTransformFunc: Vector => Vector = { vec => val result = vec.toArray val jTransformer = new DoubleDCT_1D(result.length) - if (dct.$(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) + if ($(inverse)) jTransformer.inverse(result, true) else jTransformer.forward(result, true) Vectors.dense(result) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index 626a6457f027..1b0a9a12e83b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -49,11 +49,10 @@ class ElementwiseProduct(override val uid: String) /** @group getParam */ def getScalingVec: Vector = getOrDefault(scalingVec) - override protected val createTransformFunc: (ElementwiseProduct, Vector) => Vector = { - (transformer, vec) => - require(transformer.params.contains(scalingVec), s"transformation requires a weight vector") - val elemScaler = new feature.ElementwiseProduct(transformer.$(scalingVec)) - elemScaler.transform(vec) + override protected def createTransformFunc: Vector => Vector = { + require(params.contains(scalingVec), s"transformation requires a weight vector") + val elemScaler = new feature.ElementwiseProduct($(scalingVec)) + elemScaler.transform } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala index d638d8491f1a..f8bc7e3f0c03 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/NGram.scala @@ -56,9 +56,8 @@ class NGram(override val uid: String) setDefault(n -> 2) - override protected val createTransformFunc: (NGram, Seq[String]) => Seq[String] = { - (ngram, strings) => - strings.iterator.sliding(ngram.$(n)).withPartial(false).map(_.mkString(" ")).toSeq + override protected def createTransformFunc: Seq[String] => Seq[String] = { + _.iterator.sliding($(n)).withPartial(false).map(_.mkString(" ")).toSeq } override protected def validateInputType(inputType: DataType): Unit = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index a366868586e9..a603b3f83320 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -50,10 +50,9 @@ class Normalizer(override val uid: String) /** @group setParam */ def setP(value: Double): this.type = set(p, value) - override protected val createTransformFunc: (Normalizer, Vector) => Vector = { - (transformer, vec) => - val normalizer = new feature.Normalizer(transformer.$(p)) - normalizer.transform(vec) + override protected def createTransformFunc: Vector => Vector = { + val normalizer = new feature.Normalizer($(p)) + normalizer.transform } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala index ec8f3d8cb4e5..0a9b9719c15d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PolynomialExpansion.scala @@ -56,8 +56,8 @@ class PolynomialExpansion(override val uid: String) /** @group setParam */ def setDegree(value: Int): this.type = set(degree, value) - override protected val createTransformFunc: (PolynomialExpansion, Vector) => Vector = { (p, v) => - PolynomialExpansion.expand(v, p.$(degree)) + override protected def createTransformFunc: Vector => Vector = { v => + PolynomialExpansion.expand(v, $(degree)) } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 2f6448da49bb..8456a0e91580 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -35,8 +35,8 @@ class Tokenizer(override val uid: String) def this() = this(Identifiable.randomUID("tok")) - override protected val createTransformFunc: (Tokenizer, String) => Seq[String] = { (_, str) => - str.toLowerCase.split("\\s") + override protected def createTransformFunc: String => Seq[String] = { + _.toLowerCase.split("\\s") } override protected def validateInputType(inputType: DataType): Unit = { @@ -124,13 +124,12 @@ class RegexTokenizer(override val uid: String) setDefault(minTokenLength -> 1, gaps -> true, pattern -> "\\s+", toLowercase -> true) - override protected val createTransformFunc: (RegexTokenizer, String) => Seq[String] = { - (tokenizer, originStr) => - val re = tokenizer.$(pattern).r - val str = if (tokenizer.$(toLowercase)) originStr.toLowerCase() else originStr - val tokens = if (tokenizer.$(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq - val minLength = tokenizer.$(minTokenLength) - tokens.filter(_.length >= minLength) + override protected def createTransformFunc: String => Seq[String] = { originStr => + val re = $(pattern).r + val str = if ($(toLowercase)) originStr.toLowerCase() else originStr + val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq + val minLength = $(minTokenLength) + tokens.filter(_.length >= minLength) } override protected def validateInputType(inputType: DataType): Unit = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index d38a2bb61792..efd00577332a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -177,27 +177,6 @@ case class ScalaUDF( """ } - lazy val inputEncoder: ExpressionEncoder[Row] = RowEncoder(inputSchema) - lazy val outputEncoder: ExpressionEncoder[Row] = - RowEncoder(StructType(StructField("_c0", dataType) :: Nil)) - - lazy val reflectedFunc = - runtimeMirror(function.getClass.getClassLoader).reflect(function) - lazy val applyMethods = reflectedFunc.symbol.typeSignature.member(newTermName("apply")) - .asTerm.alternatives - lazy val invokeMethod = reflectedFunc.reflectMethod(applyMethods(0).asMethod) - - override def eval(input: InternalRow): Any = { - val projected = InternalRow.fromSeq(children.map(_.eval(input))) - val cRow = inputEncoder.fromRow(projected) - - try { - val callRet = invokeMethod.apply(cRow.toSeq: _*) - outputEncoder.toRow(Row(callRet)).copy().asInstanceOf[InternalRow].get(0, dataType) - } catch { - // When exception is thrown in UDF, an InvocationTargetException will be thrown. - // We get and re-throw the cause exception. - case e: InvocationTargetException => throw e.getTargetException - } - } + override def eval(input: InternalRow): Any = + throw new UnsupportedOperationException("Only code-generated evaluation is supported.") } From 30a867e5a1910d9f5fbf43924556aee2d0694bbb Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 11 Apr 2016 02:58:32 +0000 Subject: [PATCH 25/25] Remove unnecessary import. --- .../org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index efd00577332a..de1173b1fc3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -17,10 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import java.lang.reflect.InvocationTargetException - -import scala.reflect.runtime.universe._ - import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.codegen._