aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorWenchen Fan <cloud0fan@outlook.com>2015-05-17 16:51:57 -0700
committerYin Huai <yhuai@databricks.com>2015-05-17 16:51:57 -0700
commit2f22424e9f6624097b292cb70e00787b69d80718 (patch)
tree71bafe06c7325396e66fcd8350d4db0461988a93 /sql
parentca4257aec658aaa87f4f097dd7534033d5f13ddc (diff)
downloadspark-2f22424e9f6624097b292cb70e00787b69d80718.tar.gz
spark-2f22424e9f6624097b292cb70e00787b69d80718.tar.bz2
spark-2f22424e9f6624097b292cb70e00787b69d80718.zip
[SQL] [MINOR] use catalyst type converter in ScalaUdf
It's a follow-up of https://github.com/apache/spark/pull/5154, we can speed up scala udf evaluation by create type converter in advance. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #6182 from cloud-fan/tmp and squashes the following commits: 241cfe9 [Wenchen Fan] use converter in ScalaUdf
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala4
1 files changed, 2 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 9a77ca624e..d22eb10ad3 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
@@ -956,7 +956,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi
}
// scalastyle:on
-
- override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType)
+ val converter = CatalystTypeConverters.createToCatalystConverter(dataType)
+ override def eval(input: Row): Any = converter(f(input))
}