From 0f7d50cdf6f6c760e7c2665911f952ec66a4a5cb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 23 Jul 2014 14:58:16 -0700 Subject: [PATCH 01/14] Draft of native Spark SQL UDFs for Scala and Python. --- python/pyspark/sql.py | 15 +- .../catalyst/analysis/FunctionRegistry.scala | 17 +- .../sql/catalyst/expressions/ScalaUdf.scala | 305 ++++++++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 13 +- .../spark/sql/hive/HiveStrategies.scala | 2 + .../spark/sql/hive/UdfRegistration.scala | 193 +++++++++++ .../org/apache/spark/sql/hive/hiveUdfs.scala | 5 +- .../apache/spark/sql/hive/pythonUdfs.scala | 190 +++++++++++ .../org/apache/spark/sql/PythonUDFSuite.scala | 107 ++++++ .../org/apache/spark/sql/QueryTest.scala | 2 +- 10 files changed, 838 insertions(+), 11 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index cb83e89176823..944671af8dc53 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -16,9 +16,13 @@ # from pyspark.rdd import RDD, PipelinedRDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer + +from itertools import chain, ifilter, imap from py4j.protocol import Py4JError +from py4j.java_collections import ListConverter, MapConverter + __all__ = ["SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", "SchemaRDD", "Row"] @@ -73,6 +77,15 @@ def _ssql_ctx(self): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext + def registerFunction(self, name, f, returnType = "string"): + def func(split, iterator): return imap(f, iterator) + command = (func, self._sc.serializer, self._sc.serializer) + env = MapConverter().convert(self._sc.environment, + self._sc._gateway._gateway_client) + includes = ListConverter().convert(self._sc._python_includes, + self._sc._gateway._gateway_client) + self._ssql_ctx.registerPython(name, bytearray(CloudPickleSerializer().dumps(command)), env, includes, self._sc.pythonExec, self._sc._javaAccumulator, returnType) + def inferSchema(self, rdd): """Infer and apply a schema to an RDD of L{dict}s. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c0255701b7ba5..062a62503ee28 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -18,12 +18,27 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Expression +import scala.collection.mutable /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ trait FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression } +trait OverrideFunctionRegistry extends FunctionRegistry { + type FunctionBuilder = Seq[Expression] => Expression + + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + abstract override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders.get(name).map(_(children)).getOrElse(super.lookupFunction(name,children)) + } +} + /** * A trivial catalog that returns an error when a function is requested. Used for testing when all * functions are already filled in and the analyser needs only to resolve attribute references. @@ -32,4 +47,4 @@ object EmptyFunctionRegistry extends FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } -} +} \ No newline at end of file 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 5e089f7618e0a..f0abb2d1416bb 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 @@ -27,6 +27,21 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true + /** 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"children($x).eval(input)").reduce(_ + ",\n " + _) + + s""" + case $x => + function.asInstanceOf[($anys) => Any]( + $evals) + """ + } + + */ + override def eval(input: Row): Any = { children.size match { case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) @@ -34,6 +49,296 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi function.asInstanceOf[(Any, Any) => Any]( children(0).eval(input), children(1).eval(input)) + case 3 => + function.asInstanceOf[(Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input)) + case 4 => + function.asInstanceOf[(Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input)) + case 5 => + function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input)) + case 6 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input)) + case 7 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input)) + case 8 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input)) + case 9 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input)) + case 10 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input)) + case 11 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input)) + case 12 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input)) + case 13 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input)) + case 14 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input)) + case 15 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input)) + case 16 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input)) + case 17 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input)) + case 18 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input)) + case 19 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input)) + case 20 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input)) + case 21 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input)) + case 22 => + 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]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input), + children(21).eval(input)) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 84d43eaeea51d..69d41637f1c5b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,7 +23,7 @@ import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag +import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver @@ -35,7 +35,8 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{OverrideFunctionRegistry, Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.QueryExecutionException @@ -65,7 +66,7 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. */ -class HiveContext(sc: SparkContext) extends SQLContext(sc) { +class HiveContext(sc: SparkContext) extends SQLContext(sc) with UdfRegistration{ self => override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = @@ -152,10 +153,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } + protected[sql] val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry + /* An analyzer that uses the Hive metastore. */ @transient override protected[sql] lazy val analyzer = - new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) /** * Runs the specified SQL query using Hive. @@ -247,7 +250,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = - optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed))) + optimizer(ExtractPythonUdfs(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))) override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 4d0fab4140b21..ac7e0db6340d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -34,6 +34,8 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case e @ EvaluatePython(udf, child) => + BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case logical.ScriptTransformation(input, script, output, child) => ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil case _ => Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala new file mode 100644 index 0000000000000..6430fb03f5ebb --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.util.{List => JList, Map => JMap} + +import org.apache.spark.sql.catalyst.types.StringType + +import scala.reflect.runtime.universe.{TypeTag, typeTag} + +import org.apache.spark.Accumulator +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} + + +/** + * Functions for registering UDFs. + * + * TODO: Move to SQL. + */ +protected[sql] trait UdfRegistration { + self: HiveContext => + + private[spark] def registerPython( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + dataType: String): Unit = { + println( + s""" + | Registering new PythonUDF: + | name: $name + | command: ${command.toSeq} + | envVars: $envVars + | pythonIncludes: $pythonIncludes + | pythonExec: $pythonExec + """.stripMargin) + + + def builder(e: Seq[Expression]) = + PythonUDF(name, + command, + envVars, + pythonIncludes, + pythonExec, + accumulator, + HiveMetastoreTypes.toDataType(dataType), + e) + + functionRegistry.registerFunction(name, builder) + } + + /** registerFunction 1-22 were generated by this script + + (1 to 22).map { x => + val types = (1 to x).map(x => "_").reduce(_ + ", " + _) + s""" + def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + """ + } + + */ + def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } +} \ No newline at end of file diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 057eb60a02612..ce7e173b3db20 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils.getContextOrSparkClassLoader /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { +private[hive] class HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) @@ -92,9 +92,8 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu } private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf { + extends HiveUdf with HiveInspectors { - import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @transient diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala new file mode 100644 index 0000000000000..31f728ff7ac6f --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala @@ -0,0 +1,190 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.hive + +import java.util.{List => JList, Map => JMap} + +import org.apache.spark.annotation.DeveloperApi + +import scala.collection.JavaConversions._ + +import net.razorvine.pickle.{Pickler, Unpickler} + +import org.apache.spark.{Logging, Accumulator} +import org.apache.spark.api.python.PythonRDD +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.SparkPlan + +/** + * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. + */ +private[spark] case class PythonUDF( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + dataType: DataType, + children: Seq[Expression]) extends Expression with Logging { + + override def toString = s"PythonUDF#$name(${children.mkString(",")})" + + def nullable: Boolean = true + def references: Set[Attribute] = children.flatMap(_.references).toSet + + def batchEval(input: RDD[Row]): RDD[Row] = { + val parent = input.mapPartitions { iter => + val pickle = new Pickler + val currentRow = MutableProjection(children) + iter.map { inputRow => + val toBePickled = currentRow(inputRow) + log.debug(s"toBePickled: $toBePickled") + if(children.length == 1) { + pickle.dumps(toBePickled.toArray) + } else { + pickle.dumps(Array(toBePickled.toArray)) + } + } + }.asInstanceOf[RDD[Any]] + + val pyRDD = new PythonRDD[Any]( + parent, + command, + envVars, + pythonIncludes, + false, + pythonExec, + Seq[Broadcast[Array[Byte]]](), + accumulator + ).mapPartitions { iter => + val pickle = new Unpickler + iter.flatMap { pickedResult => + val res = pickle.loads(pickedResult) + log.debug(s"pickleOutput: $res") + res.asInstanceOf[java.util.ArrayList[Any]] + } + }.mapPartitions { iter => + val row = new GenericMutableRow(1) + iter.map { result => + row(0) = dataType match { + case StringType => result.toString + case other => result + } + log.debug(s"resultRow: $row") + row: Row + } + } + + pyRDD + } + + override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") +} + +/** + * Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated + * alone in a batch. + * + * This has the limitation that the input to the Python UDF is not allowed include attributes from + * multiple child operators. + */ +private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan) = plan transform { + // Skip EvaluatePython nodes. + case p: EvaluatePython => p + + case l: LogicalPlan => + // Extract any PythonUDFs from the current operator. + val udfs = l.expressions.flatMap(_.collect { case udf: PythonUDF => udf}) + if (udfs.isEmpty) { + // If there aren't any, we are done. + l + } else { + // Pick the UDF we are going to evaluate (TODO: Support evaluating multiple UDFs at a time) + // If there is more than one, we will add another evaluation operator in a subsequent pass. + val udf = udfs.head + + var evaluation: EvaluatePython = null + + // Rewrite the child that has the input required for the UDF + val newChildren = l.children.map { child => + if (udf.references.subsetOf(child.outputSet)) { + evaluation = EvaluatePython(udf, child) + evaluation + } else if (udf.references.intersect(child.outputSet).nonEmpty) { + sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") + } else { + child + } + } + + assert(evaluation != null, "Unable to evaluate PythonUDF. Missing input attributes.") + + // Trim away the new UDF value if it was only used for filtering or something. + Project( + l.output, + l.transformExpressions { + case p: PythonUDF if p.id == udf.id => evaluation.resultAttribute + }.withNewChildren(newChildren)) + } + } +} + +/** + * :: DeveloperApi :: + * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. + */ +@DeveloperApi +case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { + val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() + + def references = Set.empty + def output = child.output :+ resultAttribute +} + +/** + * :: DeveloperApi :: + * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. The input + * data is cached and zipped with the result of the udf evaluation. + */ +@DeveloperApi +case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) + extends SparkPlan { + def children = child :: Nil + + def execute() = { + // TODO: Clean up after ourselves? + val childResults = child.execute().map(_.copy()).cache() + + childResults.zip(udf.batchEval(childResults)).mapPartitions { iter => + val joinedRow = new JoinedRow() + iter.map { + case (row, udfResult) => + joinedRow(row, udfResult) + } + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala new file mode 100644 index 0000000000000..9337b6bea10fb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.catalyst.expressions.GenericRow + +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.types._ + +class PythonUDFSuite extends QueryTest { + import org.apache.spark.sql.hive.test.TestHive._ + + val testData = sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))) + testData.registerAsTable("testData") + + /* sqlCtx.registerFunction("test", lambda (x): "test:" + x) */ + registerPython( + "test", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, 124, 0, 0, 23, 83, 113, 27, 78, 85, 5, 116, 101, 115, 116, 58, 113, 28, -122, 113, 29, 41, 85, 1, 120, 113, 30, -123, 113, 31, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 32, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 33, 75, 1, 85, 0, 113, 34, 41, 41, 116, 113, 35, 82, 113, 36, 75, 0, 125, 113, 37, -121, 113, 38, 82, 113, 39, 125, 113, 40, 78, 93, 113, 41, 125, 113, 42, 116, 82, 97, 125, 113, 43, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 44, 41, -127, 113, 45, 125, 113, 46, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 47, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 48, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 49, 41, -127, 113, 50, 125, 113, 51, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 52, -119, 115, 98, 117, 98, 104, 45, -121, 113, 53, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "string" + ) + + /** + * import re + * sqlCtx.registerFunction("countMatches", lambda (pattern, string): re.subn(pattern, '', string)[1], returnType = "int") + */ + registerPython( + "countMatches", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 3, 75, 4, 75, 67, 85, 35, 124, 0, 0, 92, 2, 0, 125, 1, 0, 125, 2, 0, 116, 0, 0, 106, 1, 0, 124, 1, 0, 100, 1, 0, 124, 2, 0, -125, 3, 0, 100, 2, 0, 25, 83, 113, 27, 78, 104, 16, 75, 1, -121, 113, 28, 85, 2, 114, 101, 113, 29, 85, 4, 115, 117, 98, 110, 113, 30, -122, 113, 31, 85, 2, 46, 48, 113, 32, 85, 7, 112, 97, 116, 116, 101, 114, 110, 113, 33, 85, 6, 115, 116, 114, 105, 110, 103, 113, 34, -121, 113, 35, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 36, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 37, 75, 1, 85, 2, 3, 0, 113, 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, 41, -121, 113, 42, 82, 113, 43, 125, 113, 44, 104, 29, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 115, 117, 98, 105, 109, 112, 111, 114, 116, 10, 113, 45, 85, 2, 114, 101, 113, 46, -123, 113, 47, 82, 113, 48, 115, 78, 93, 113, 49, 125, 113, 50, 116, 82, 97, 125, 113, 51, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 52, 41, -127, 113, 53, 125, 113, 54, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 55, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 56, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 57, 41, -127, 113, 58, 125, 113, 59, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 60, -119, 115, 98, 117, 98, 104, 53, -121, 113, 61, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "int" + ) + + /** + * sqlCtx.registerFunction("len", lambda str: len(str), returnType = "int") + */ + registerPython( + "len", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, 124, 0, 0, -125, 1, 0, 83, 113, 27, 78, -123, 113, 28, 85, 3, 108, 101, 110, 113, 29, -123, 113, 30, 85, 3, 115, 116, 114, 113, 31, -123, 113, 32, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 33, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 34, 75, 1, 85, 0, 113, 35, 41, 41, 116, 113, 36, 82, 113, 37, 75, 0, 125, 113, 38, -121, 113, 39, 82, 113, 40, 125, 113, 41, 78, 93, 113, 42, 125, 113, 43, 116, 82, 97, 125, 113, 44, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 45, 41, -127, 113, 46, 125, 113, 47, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 48, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 50, 41, -127, 113, 51, 125, 113, 52, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 53, -119, 115, 98, 117, 98, 104, 46, -121, 113, 54, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "int" + ) + + test("Single argument UDF") { + checkAnswer( + hql("SELECT test(value) FROM testData"), + testData.select('value).map(r => + new GenericRow(Array[Any]("test:" + r(0).toString))).collect().toSeq) + } + + test("Multiple argument UDF") { + checkAnswer( + hql("SELECT countMatches('1', value) FROM testData"), + testData.select('value).map(r => + new GenericRow(Array[Any]( + "1".r.findAllMatchIn(r(0).toString).length))).collect().toSeq) + } + + test("UDF in WHERE") { + checkAnswer( + hql("SELECT value FROM testData WHERE countMatches('1', value) >= 1"), + testData + .select('value) + .filter(r => "1".r.findAllMatchIn(r(0).toString).length >= 1) + .collect().toSeq) + } + + test("Integer Return Type") { + sparkContext.parallelize(1 to 20).map(i => new StringData("*" * i)).registerAsTable("stars") + checkAnswer( + hql("SELECT len(str) FROM stars"), + table("stars").map(r => + new GenericRow(Array[Any](r.getString(0).length))).collect().toSeq + ) + + } +} + +case class StringData(str: String) \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 11d8b1f0a3d96..5a9300b0e00e6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -51,7 +51,7 @@ class QueryTest extends FunSuite { fail( s""" |Exception thrown while executing query: - |${rdd.logicalPlan} + |${rdd.queryExecution} |== Exception == |$e """.stripMargin) From 2766f0b0cae471bc9e5b047df4966440786967cf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 27 Jul 2014 18:27:38 -0700 Subject: [PATCH 02/14] Move udfs support to SQL from hive. Add support for Java UDFs. --- .../catalyst/analysis/FunctionRegistry.scala | 17 +- .../sql/catalyst/expressions/ScalaUdf.scala | 4 +- .../org/apache/spark/sql/api/java/UDF1.java | 33 +++ .../org/apache/spark/sql/api/java/UDF10.java | 33 +++ .../org/apache/spark/sql/api/java/UDF11.java | 33 +++ .../org/apache/spark/sql/api/java/UDF12.java | 33 +++ .../org/apache/spark/sql/api/java/UDF13.java | 33 +++ .../org/apache/spark/sql/api/java/UDF14.java | 33 +++ .../org/apache/spark/sql/api/java/UDF15.java | 33 +++ .../org/apache/spark/sql/api/java/UDF16.java | 33 +++ .../org/apache/spark/sql/api/java/UDF17.java | 33 +++ .../org/apache/spark/sql/api/java/UDF18.java | 33 +++ .../org/apache/spark/sql/api/java/UDF19.java | 33 +++ .../org/apache/spark/sql/api/java/UDF2.java | 33 +++ .../org/apache/spark/sql/api/java/UDF20.java | 33 +++ .../org/apache/spark/sql/api/java/UDF21.java | 33 +++ .../org/apache/spark/sql/api/java/UDF22.java | 33 +++ .../org/apache/spark/sql/api/java/UDF3.java | 33 +++ .../org/apache/spark/sql/api/java/UDF4.java | 33 +++ .../org/apache/spark/sql/api/java/UDF5.java | 33 +++ .../org/apache/spark/sql/api/java/UDF6.java | 33 +++ .../org/apache/spark/sql/api/java/UDF7.java | 33 +++ .../org/apache/spark/sql/api/java/UDF8.java | 33 +++ .../org/apache/spark/sql/api/java/UDF9.java | 33 +++ .../org/apache/spark/sql/SQLContext.scala | 9 +- .../apache/spark/sql}/UdfRegistration.scala | 27 ++- .../sql/api/java/FunctionRegistration.scala | 227 ++++++++++++++++++ .../spark/sql/api/java/JavaSQLContext.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 2 + .../spark/sql/execution}/pythonUdfs.scala | 24 +- .../spark/sql/api/java/JavaAPISuite.java | 94 ++++++++ .../apache/spark/sql/InsertIntoSuite.scala | 2 +- .../org/apache/spark/sql/PythonUDFSuite.scala | 199 +++++++++++++++ .../org/apache/spark/sql/QueryTest.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 5 +- .../spark/sql/hive/HiveStrategies.scala | 2 - .../org/apache/spark/sql/PythonUDFSuite.scala | 107 --------- .../org/apache/spark/sql/QueryTest.scala | 2 +- 38 files changed, 1304 insertions(+), 149 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java rename sql/{hive/src/main/scala/org/apache/spark/sql/hive => core/src/main/scala/org/apache/spark/sql}/UdfRegistration.scala (94%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala rename sql/{hive/src/main/scala/org/apache/spark/sql/hive => core/src/main/scala/org/apache/spark/sql/execution}/pythonUdfs.scala (94%) create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 062a62503ee28..c3a4ff51a9bd8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -22,11 +22,12 @@ import scala.collection.mutable /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ trait FunctionRegistry { + type FunctionBuilder = Seq[Expression] => Expression + def lookupFunction(name: String, children: Seq[Expression]): Expression } trait OverrideFunctionRegistry extends FunctionRegistry { - type FunctionBuilder = Seq[Expression] => Expression val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() @@ -39,6 +40,18 @@ trait OverrideFunctionRegistry extends FunctionRegistry { } } +class SimpleFunctionRegistry extends FunctionRegistry { + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders(name)(children) + } +} + /** * A trivial catalog that returns an error when a function is requested. Used for testing when all * functions are already filled in and the analyser needs only to resolve attribute references. @@ -47,4 +60,4 @@ object EmptyFunctionRegistry extends FunctionRegistry { def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } -} \ No newline at end of file +} 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 f0abb2d1416bb..5de4ec24672a6 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 @@ -40,8 +40,9 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi """ } - */ + */ + // scalastyle:off override def eval(input: Row): Any = { children.size match { case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) @@ -340,5 +341,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi children(20).eval(input), children(21).eval(input)) } + // scalastyle:on } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java new file mode 100644 index 0000000000000..dc07b2203bc23 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 1 arguments. + */ +public interface UDF1 extends Serializable { + public R call(T1 t1) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java new file mode 100644 index 0000000000000..7358654e1c748 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 10 arguments. + */ +public interface UDF10 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java new file mode 100644 index 0000000000000..51f66b67151d0 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 11 arguments. + */ +public interface UDF11 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java new file mode 100644 index 0000000000000..daedb250b58c7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 12 arguments. + */ +public interface UDF12 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java new file mode 100644 index 0000000000000..1ef3fc2d8df1f --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 13 arguments. + */ +public interface UDF13 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java new file mode 100644 index 0000000000000..f1a872c96863e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 14 arguments. + */ +public interface UDF14 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java new file mode 100644 index 0000000000000..91ae68eb5b317 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 15 arguments. + */ +public interface UDF15 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java new file mode 100644 index 0000000000000..deb45150b6c78 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 16 arguments. + */ +public interface UDF16 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java new file mode 100644 index 0000000000000..14fd59a31181c --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 17 arguments. + */ +public interface UDF17 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java new file mode 100644 index 0000000000000..364c3e87b862d --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 18 arguments. + */ +public interface UDF18 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java new file mode 100644 index 0000000000000..1193d45124b2e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 19 arguments. + */ +public interface UDF19 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java new file mode 100644 index 0000000000000..fc77c3cf8249d --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 2 arguments. + */ +public interface UDF2 extends Serializable { + public R call(T1 t1, T2 t2) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java new file mode 100644 index 0000000000000..89d2600870dab --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 20 arguments. + */ +public interface UDF20 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java new file mode 100644 index 0000000000000..6a084b929a37e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 21 arguments. + */ +public interface UDF21 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java new file mode 100644 index 0000000000000..4d9ee070e77b4 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 22 arguments. + */ +public interface UDF22 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java new file mode 100644 index 0000000000000..eb90d5b152f48 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 3 arguments. + */ +public interface UDF3 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java new file mode 100644 index 0000000000000..cf731e334f2ec --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 4 arguments. + */ +public interface UDF4 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java new file mode 100644 index 0000000000000..5501f5e31cbd7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 5 arguments. + */ +public interface UDF5 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java new file mode 100644 index 0000000000000..b42e9aa2639c8 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 6 arguments. + */ +public interface UDF6 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java new file mode 100644 index 0000000000000..9d5d571bb413d --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 7 arguments. + */ +public interface UDF7 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java new file mode 100644 index 0000000000000..36caade21dc22 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 8 arguments. + */ +public interface UDF8 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java new file mode 100644 index 0000000000000..f934d93914daa --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 9 arguments. + */ +public interface UDF9 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; +} + \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4abd89955bd27..be0d0b1da6079 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -52,15 +52,20 @@ class SQLContext(@transient val sparkContext: SparkContext) extends Logging with SQLConf with ExpressionConversions + with UdfRegistration with Serializable { self => @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + + @transient + protected[sql] lazy val functionRegistry = new SimpleFunctionRegistry + @transient protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + new Analyzer(catalog, functionRegistry, caseSensitive = true) @transient protected[sql] val optimizer = Optimizer @transient @@ -314,7 +319,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected abstract class QueryExecution { def logical: LogicalPlan - lazy val analyzed = analyzer(logical) + lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) lazy val optimizedPlan = optimizer(analyzed) // TODO: Don't just pick the first one... lazy val sparkPlan = planner(optimizedPlan).next() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala similarity index 94% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala rename to sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 6430fb03f5ebb..5169fbe104fa8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -15,26 +15,23 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql import java.util.{List => JList, Map => JMap} -import org.apache.spark.sql.catalyst.types.StringType - -import scala.reflect.runtime.universe.{TypeTag, typeTag} - import org.apache.spark.Accumulator import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.execution.PythonUDF + +import scala.reflect.runtime.universe.{TypeTag, typeTag} /** - * Functions for registering UDFs. - * - * TODO: Move to SQL. + * Functions for registering UDFs in a context. */ protected[sql] trait UdfRegistration { - self: HiveContext => + self: SQLContext => private[spark] def registerPython( name: String, @@ -44,7 +41,7 @@ protected[sql] trait UdfRegistration { pythonExec: String, accumulator: Accumulator[JList[Array[Byte]]], dataType: String): Unit = { - println( + logger.debug( s""" | Registering new PythonUDF: | name: $name @@ -62,7 +59,8 @@ protected[sql] trait UdfRegistration { pythonIncludes, pythonExec, accumulator, - HiveMetastoreTypes.toDataType(dataType), + org.apache.spark.sql.catalyst.types.StringType, + // FIXME: HiveMetastoreTypes.toDataType(dataType), e) functionRegistry.registerFunction(name, builder) @@ -74,13 +72,15 @@ protected[sql] trait UdfRegistration { val types = (1 to x).map(x => "_").reduce(_ + ", " + _) s""" def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = + ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) functionRegistry.registerFunction(name, builder) } """ } + */ - */ + // scalastyle:off def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) functionRegistry.registerFunction(name, builder) @@ -190,4 +190,5 @@ protected[sql] trait UdfRegistration { def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) functionRegistry.registerFunction(name, builder) } + // scalastyle:on } \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala new file mode 100644 index 0000000000000..e8b1f4f1962ae --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala @@ -0,0 +1,227 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.catalyst.types.DataType + +/** + * A collection of functions that allow Java users to register UDFs. In order to handle functions + * of varying airities with minimal boilerplate for our users, we generate classes and functions + * for each airity up to 22. The code for this generation can be found in comments in this trait. + */ +private[java] trait FunctionRegistration { + self: JavaSQLContext => + + /* The following functions and required interfaces are generated with these code fragments: + + (1 to 22).foreach { i => + val extTypeArgs = (1 to i).map(_ => "_").mkString(", ") + val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ") + val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" + val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") + println(s""" + |def registerFunction(name: String, f: UDF$i[$extTypeArgs, _], dataType: DataType) = + | sqlContext.functionRegistry.registerFunction( + | name, + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), dataType, e)) + """.stripMargin) + } + + import java.io.File + import org.apache.spark.sql.catalyst.util.stringToFile + val directory = new File("sql/core/src/main/java/org/apache/spark/sql/api/java/") + (1 to 22).foreach { i => + val typeArgs = (1 to i).map(i => s"T$i").mkString(", ") + val args = (1 to i).map(i => s"T$i t$i").mkString(", ") + + val contents = + s"""/* + | * Licensed to the Apache Software Foundation (ASF) under one or more + | * contributor license agreements. See the NOTICE file distributed with + | * this work for additional information regarding copyright ownership. + | * The ASF licenses this file to You under the Apache License, Version 2.0 + | * (the "License"); you may not use this file except in compliance with + | * the License. You may obtain a copy of the License at + | * + | * http://www.apache.org/licenses/LICENSE-2.0 + | * + | * Unless required by applicable law or agreed to in writing, software + | * distributed under the License is distributed on an "AS IS" BASIS, + | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + | * See the License for the specific language governing permissions and + | * limitations under the License. + | */ + | + |package org.apache.spark.sql.api.java; + | + |import java.io.Serializable; + | + |// ************************************************** + |// THIS FILE IS AUTOGENERATED BY CODE IN + |// org.apache.spark.sql.api.java.FunctionRegistration + |// ************************************************** + | + |/** + | * A Spark SQL UDF that has $i arguments. + | */ + |public interface UDF$i<$typeArgs, R> extends Serializable { + | public R call($args) throws Exception; + |} + """.stripMargin + + stringToFile(new File(directory, s"UDF$i.java"), contents) + } + + */ + + // scalastyle:off + def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + + + // scalastyle:on +} \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 790d9ef22cf16..21c307eaf4079 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -25,7 +25,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} +import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow, Expression, ScalaUdf, AttributeReference, GenericRow} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(val sqlContext: SQLContext) { +class JavaSQLContext(val sqlContext: SQLContext) extends FunctionRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c078e71fe0290..a61e96368160b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -283,6 +283,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + case e @ EvaluatePython(udf, child) => + BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case SparkLogicalPlan(existingPlan, _) => existingPlan :: Nil case _ => Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala similarity index 94% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 31f728ff7ac6f..b9c7458bd3676 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -15,27 +15,23 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} -import org.apache.spark.annotation.DeveloperApi - -import scala.collection.JavaConversions._ - import net.razorvine.pickle.{Pickler, Unpickler} - -import org.apache.spark.{Logging, Accumulator} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonRDD import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD - import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.{Accumulator, Logging => SparkLogging} + +import scala.collection.JavaConversions._ /** * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. @@ -48,7 +44,7 @@ private[spark] case class PythonUDF( pythonExec: String, accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, - children: Seq[Expression]) extends Expression with Logging { + children: Seq[Expression]) extends Expression with SparkLogging { override def toString = s"PythonUDF#$name(${children.mkString(",")})" @@ -70,7 +66,7 @@ private[spark] case class PythonUDF( } }.asInstanceOf[RDD[Any]] - val pyRDD = new PythonRDD[Any]( + val pyRDD = new PythonRDD( parent, command, envVars, @@ -144,7 +140,7 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { assert(evaluation != null, "Unable to evaluate PythonUDF. Missing input attributes.") // Trim away the new UDF value if it was only used for filtering or something. - Project( + logical.Project( l.output, l.transformExpressions { case p: PythonUDF if p.id == udf.id => evaluation.resultAttribute diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java new file mode 100644 index 0000000000000..c8074fd4b3782 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +import org.apache.spark.sql.api.java.UDF1; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Suite; +import org.junit.runner.RunWith; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.api.java.JavaSQLContext; + +// TODO: USE YIN'S API HERE +import org.apache.spark.sql.catalyst.types.IntegerType$; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite implements Serializable { + private transient JavaSparkContext sc; + private transient JavaSQLContext sqlContext; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + sqlContext = new JavaSQLContext(sc); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @SuppressWarnings("unchecked") + @Test + public void udf1Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", (String str) -> str.length(), IntegerType$.MODULE$); + + sqlContext.registerFunction("stringLengthTest", new UDF1() { + @Override + public Integer call(String str) throws Exception { + return str.length(); + } + }, IntegerType$.MODULE$); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); + assert(result.getInt(0) == 4); + } + + @SuppressWarnings("unchecked") + @Test + public void udf2Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", + // (String str1, String str2) -> str1.length() + str2.length, + // IntegerType$.MODULE$); + + sqlContext.registerFunction("stringLengthTest", new UDF2() { + @Override + public Integer call(String str1, String str2) throws Exception { + return str1.length() + str2.length(); + } + }, IntegerType$.MODULE$); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); + assert(result.getInt(0) == 9); + } +} \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 4f0b85f26254b..23a711d08c58b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.File +import _root_.java.io.File /* Implicits */ import org.apache.spark.sql.test.TestSQLContext._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala new file mode 100644 index 0000000000000..e7ecf40e952e2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.test.TestSQLContext + +class PythonUDFSuite extends QueryTest { + import TestSQLContext._ + + val testData = sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))) + testData.registerAsTable("testData") + + /* sqlCtx.registerFunction("test", lambda (x): "test:" + x) */ + registerPython( + "test", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, + 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, + 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, + 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, + 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, + 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, + 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, + 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, + 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, + 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, + 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, + 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, + 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, + 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, + 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, + 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, + 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, + 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, 124, 0, 0, 23, 83, 113, 27, 78, + 85, 5, 116, 101, 115, 116, 58, 113, 28, -122, 113, 29, 41, 85, 1, 120, 113, 30, -123, 113, 31, + 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 32, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, + 113, 33, 75, 1, 85, 0, 113, 34, 41, 41, 116, 113, 35, 82, 113, 36, 75, 0, 125, 113, 37, -121, + 113, 38, 82, 113, 39, 125, 113, 40, 78, 93, 113, 41, 125, 113, 42, 116, 82, 97, 125, 113, 43, + 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, + 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, + 10, 113, 44, 41, -127, 113, 45, 125, 113, 46, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, + 101, 113, 47, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 48, 99, + 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, + 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 49, + 41, -127, 113, 50, 125, 113, 51, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, + 95, 115, 116, 114, 105, 110, 103, 115, 113, 52, -119, 115, 98, 117, 98, 104, 45, -121, 113, + 53, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "string" + ) + + /** + * import re + * sqlCtx.registerFunction("countMatches", + * lambda (pattern, string): re.subn(pattern, '', string)[1], returnType = "int") + */ + registerPython( + "countMatches", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, + 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, + 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, + 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, + 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, + 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, + 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, + 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, + 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, + 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, + 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, + 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, + 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, + 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, + 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, + 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, + 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, + 5, 104, 6, 40, 75, 1, 75, 3, 75, 4, 75, 67, 85, 35, 124, 0, 0, 92, 2, 0, 125, 1, 0, 125, 2, 0, + 116, 0, 0, 106, 1, 0, 124, 1, 0, 100, 1, 0, 124, 2, 0, -125, 3, 0, 100, 2, 0, 25, 83, 113, 27, + 78, 104, 16, 75, 1, -121, 113, 28, 85, 2, 114, 101, 113, 29, 85, 4, 115, 117, 98, 110, 113, + 30, -122, 113, 31, 85, 2, 46, 48, 113, 32, 85, 7, 112, 97, 116, 116, 101, 114, 110, 113, 33, + 85, 6, 115, 116, 114, 105, 110, 103, 113, 34, -121, 113, 35, 85, 7, 60, 115, 116, 100, 105, + 110, 62, 113, 36, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 37, 75, 1, 85, 2, 3, 0, 113, + 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, 41, -121, 113, 42, 82, 113, 43, 125, + 113, 44, 104, 29, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, + 99, 107, 108, 101, 10, 115, 117, 98, 105, 109, 112, 111, 114, 116, 10, 113, 45, 85, 2, 114, + 101, 113, 46, -123, 113, 47, 82, 113, 48, 115, 78, 93, 113, 49, 125, 113, 50, 116, 82, 97, + 125, 113, 51, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, + 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, + 122, 101, 114, 10, 113, 52, 41, -127, 113, 53, 125, 113, 54, 40, 85, 9, 98, 97, 116, 99, 104, + 83, 105, 122, 101, 113, 55, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, + 113, 56, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, + 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, + 113, 57, 41, -127, 113, 58, 125, 113, 59, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, + 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 60, -119, 115, 98, 117, 98, 104, 53, + -121, 113, 61, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "int" + ) + + /** + * sqlCtx.registerFunction("len", lambda str: len(str), returnType = "int") + */ + registerPython( + "len", + Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, + 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, + 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, + 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, + 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, + 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, + 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, + 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, + 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, + 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, + 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, + 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, + 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, + 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, + 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, + 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, + 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, + 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, 124, 0, 0, -125, 1, 0, 83, 113, + 27, 78, -123, 113, 28, 85, 3, 108, 101, 110, 113, 29, -123, 113, 30, 85, 3, 115, 116, 114, + 113, 31, -123, 113, 32, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 33, 85, 8, 60, 108, 97, + 109, 98, 100, 97, 62, 113, 34, 75, 1, 85, 0, 113, 35, 41, 41, 116, 113, 36, 82, 113, 37, 75, + 0, 125, 113, 38, -121, 113, 39, 82, 113, 40, 125, 113, 41, 78, 93, 113, 42, 125, 113, 43, 116, + 82, 97, 125, 113, 44, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, + 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, + 108, 105, 122, 101, 114, 10, 113, 45, 41, -127, 113, 46, 125, 113, 47, 40, 85, 9, 98, 97, 116, + 99, 104, 83, 105, 122, 101, 113, 48, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, + 101, 114, 113, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, + 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, + 114, 10, 113, 50, 41, -127, 113, 51, 125, 113, 52, 85, 19, 95, 111, 110, 108, 121, 95, 119, + 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 53, -119, 115, 98, 117, 98, + 104, 46, -121, 113, 54, 46), + new java.util.HashMap[String, String](), + new java.util.LinkedList[String](), + "python", + null, + "int" + ) + + test("Single argument UDF") { + checkAnswer( + sql("SELECT test(value) FROM testData"), + testData.select('value).map(r => + new GenericRow(Array[Any]("test:" + r(0).toString))).collect().toSeq) + } + + test("Multiple argument UDF") { + checkAnswer( + sql("SELECT countMatches('1', value) FROM testData"), + testData.select('value).map(r => + new GenericRow(Array[Any]( + "1".r.findAllMatchIn(r(0).toString).length))).collect().toSeq) + } + + test("UDF in WHERE") { + checkAnswer( + sql("SELECT value FROM testData WHERE countMatches('1', value) >= 1"), + testData + .select('value) + .filter(r => "1".r.findAllMatchIn(r(0).toString).length >= 1) + .collect().toSeq) + } + + test("Integer Return Type") { + sparkContext.parallelize(1 to 20).map(i => new StringData("*" * i)).registerAsTable("stars") + checkAnswer( + sql("SELECT len(str) FROM stars"), + table("stars").map(r => + new GenericRow(Array[Any](r.getString(0).length))).collect().toSeq + ) + + } +} + +case class StringData(str: String) \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 8e1e1971d968b..5ddcdba3944e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -44,7 +44,7 @@ class QueryTest extends PlanTest { |Exception thrown while executing query: |${rdd.queryExecution} |== Exception == - |$e + |${stackTraceToString(e)} """.stripMargin) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 69d41637f1c5b..0aa3c2c4750b0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -39,8 +39,7 @@ import org.apache.spark.sql.catalyst.analysis.{OverrideFunctionRegistry, Analyze import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.QueryExecutionException -import org.apache.spark.sql.execution.{Command => PhysicalCommand} +import org.apache.spark.sql.execution.{Command => PhysicalCommand, ExtractPythonUdfs, QueryExecutionException} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand /** @@ -66,7 +65,7 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { * An instance of the Spark SQL execution engine that integrates with data stored in Hive. * Configuration for Hive is read from hive-site.xml on the classpath. */ -class HiveContext(sc: SparkContext) extends SQLContext(sc) with UdfRegistration{ +class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index ac7e0db6340d0..4d0fab4140b21 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -34,8 +34,6 @@ private[hive] trait HiveStrategies { object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case e @ EvaluatePython(udf, child) => - BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case logical.ScriptTransformation(input, script, output, child) => ScriptTransformation(input, script, output, planLater(child))(hiveContext) :: Nil case _ => Nil diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala deleted file mode 100644 index 9337b6bea10fb..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import org.apache.spark.sql.catalyst.expressions.GenericRow - -import scala.collection.JavaConversions._ - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.catalyst.types._ - -class PythonUDFSuite extends QueryTest { - import org.apache.spark.sql.hive.test.TestHive._ - - val testData = sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") - - /* sqlCtx.registerFunction("test", lambda (x): "test:" + x) */ - registerPython( - "test", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, 124, 0, 0, 23, 83, 113, 27, 78, 85, 5, 116, 101, 115, 116, 58, 113, 28, -122, 113, 29, 41, 85, 1, 120, 113, 30, -123, 113, 31, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 32, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 33, 75, 1, 85, 0, 113, 34, 41, 41, 116, 113, 35, 82, 113, 36, 75, 0, 125, 113, 37, -121, 113, 38, 82, 113, 39, 125, 113, 40, 78, 93, 113, 41, 125, 113, 42, 116, 82, 97, 125, 113, 43, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 44, 41, -127, 113, 45, 125, 113, 46, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 47, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 48, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 49, 41, -127, 113, 50, 125, 113, 51, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 52, -119, 115, 98, 117, 98, 104, 45, -121, 113, 53, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "string" - ) - - /** - * import re - * sqlCtx.registerFunction("countMatches", lambda (pattern, string): re.subn(pattern, '', string)[1], returnType = "int") - */ - registerPython( - "countMatches", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 3, 75, 4, 75, 67, 85, 35, 124, 0, 0, 92, 2, 0, 125, 1, 0, 125, 2, 0, 116, 0, 0, 106, 1, 0, 124, 1, 0, 100, 1, 0, 124, 2, 0, -125, 3, 0, 100, 2, 0, 25, 83, 113, 27, 78, 104, 16, 75, 1, -121, 113, 28, 85, 2, 114, 101, 113, 29, 85, 4, 115, 117, 98, 110, 113, 30, -122, 113, 31, 85, 2, 46, 48, 113, 32, 85, 7, 112, 97, 116, 116, 101, 114, 110, 113, 33, 85, 6, 115, 116, 114, 105, 110, 103, 113, 34, -121, 113, 35, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 36, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 37, 75, 1, 85, 2, 3, 0, 113, 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, 41, -121, 113, 42, 82, 113, 43, 125, 113, 44, 104, 29, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 115, 117, 98, 105, 109, 112, 111, 114, 116, 10, 113, 45, 85, 2, 114, 101, 113, 46, -123, 113, 47, 82, 113, 48, 115, 78, 93, 113, 49, 125, 113, 50, 116, 82, 97, 125, 113, 51, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 52, 41, -127, 113, 53, 125, 113, 54, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 55, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 56, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 57, 41, -127, 113, 58, 125, 113, 59, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 60, -119, 115, 98, 117, 98, 104, 53, -121, 113, 61, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "int" - ) - - /** - * sqlCtx.registerFunction("len", lambda str: len(str), returnType = "int") - */ - registerPython( - "len", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, 124, 0, 0, -125, 1, 0, 83, 113, 27, 78, -123, 113, 28, 85, 3, 108, 101, 110, 113, 29, -123, 113, 30, 85, 3, 115, 116, 114, 113, 31, -123, 113, 32, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 33, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 34, 75, 1, 85, 0, 113, 35, 41, 41, 116, 113, 36, 82, 113, 37, 75, 0, 125, 113, 38, -121, 113, 39, 82, 113, 40, 125, 113, 41, 78, 93, 113, 42, 125, 113, 43, 116, 82, 97, 125, 113, 44, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 45, 41, -127, 113, 46, 125, 113, 47, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 48, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 50, 41, -127, 113, 51, 125, 113, 52, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 53, -119, 115, 98, 117, 98, 104, 46, -121, 113, 54, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "int" - ) - - test("Single argument UDF") { - checkAnswer( - hql("SELECT test(value) FROM testData"), - testData.select('value).map(r => - new GenericRow(Array[Any]("test:" + r(0).toString))).collect().toSeq) - } - - test("Multiple argument UDF") { - checkAnswer( - hql("SELECT countMatches('1', value) FROM testData"), - testData.select('value).map(r => - new GenericRow(Array[Any]( - "1".r.findAllMatchIn(r(0).toString).length))).collect().toSeq) - } - - test("UDF in WHERE") { - checkAnswer( - hql("SELECT value FROM testData WHERE countMatches('1', value) >= 1"), - testData - .select('value) - .filter(r => "1".r.findAllMatchIn(r(0).toString).length >= 1) - .collect().toSeq) - } - - test("Integer Return Type") { - sparkContext.parallelize(1 to 20).map(i => new StringData("*" * i)).registerAsTable("stars") - checkAnswer( - hql("SELECT len(str) FROM stars"), - table("stars").map(r => - new GenericRow(Array[Any](r.getString(0).length))).collect().toSeq - ) - - } -} - -case class StringData(str: String) \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 5a9300b0e00e6..95921c3d7ae09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -53,7 +53,7 @@ class QueryTest extends FunSuite { |Exception thrown while executing query: |${rdd.queryExecution} |== Exception == - |$e + |${stackTraceToString(e)} """.stripMargin) } From 6237c8d116b57bb3dfd3dbeb8bc03d76a4315484 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 28 Jul 2014 10:59:30 -0700 Subject: [PATCH 03/14] WIP --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 0aa3c2c4750b0..b5531ef9838cc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -152,7 +152,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } - protected[sql] val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry + override protected[sql] lazy val functionRegistry = + new HiveFunctionRegistry with OverrideFunctionRegistry /* An analyzer that uses the Hive metastore. */ @transient From 8e6c9329546894a0b9e71f80801ee7d6c175386c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 31 Jul 2014 20:12:30 -0700 Subject: [PATCH 04/14] WIP --- .../sql/api/java/FunctionRegistration.scala | 143 ++++++++++++------ .../spark/sql/api/java/JavaAPISuite.java | 13 +- 2 files changed, 99 insertions(+), 57 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala index e8b1f4f1962ae..4c6645919338f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.api.java import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.api.java.types.DataType +import org.apache.spark.sql.types.util.DataTypeConversions._ /** * A collection of functions that allow Java users to register UDFs. In order to handle functions @@ -36,10 +37,12 @@ private[java] trait FunctionRegistration { val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") println(s""" - |def registerFunction(name: String, f: UDF$i[$extTypeArgs, _], dataType: DataType) = + |def registerFunction(name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { + | val scalaType = asScalaDataType(dataType) | sqlContext.functionRegistry.registerFunction( | name, - | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), dataType, e)) + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e)) + |} """.stripMargin) } @@ -91,137 +94,179 @@ private[java] trait FunctionRegistration { */ // scalastyle:off - def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = + def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } - def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = + def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, - (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), dataType, e)) - - + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } // scalastyle:on } \ No newline at end of file diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index c8074fd4b3782..eff6ff951bab2 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -28,10 +28,7 @@ import org.junit.runner.RunWith; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.api.java.JavaSQLContext; - -// TODO: USE YIN'S API HERE -import org.apache.spark.sql.catalyst.types.IntegerType$; +import org.apache.spark.sql.api.java.types.DataType; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -57,14 +54,14 @@ public void tearDown() { public void udf1Test() { // With Java 8 lambdas: // sqlContext.registerFunction( - // "stringLengthTest", (String str) -> str.length(), IntegerType$.MODULE$); + // "stringLengthTest", (String str) -> str.length(), DataType.IntegerType); sqlContext.registerFunction("stringLengthTest", new UDF1() { @Override public Integer call(String str) throws Exception { return str.length(); } - }, IntegerType$.MODULE$); + }, DataType.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); @@ -78,14 +75,14 @@ public void udf2Test() { // sqlContext.registerFunction( // "stringLengthTest", // (String str1, String str2) -> str1.length() + str2.length, - // IntegerType$.MODULE$); + // DataType.IntegerType); sqlContext.registerFunction("stringLengthTest", new UDF2() { @Override public Integer call(String str1, String str2) throws Exception { return str1.length() + str2.length(); } - }, IntegerType$.MODULE$); + }, DataType.IntegerType); // TODO: Why do we need this cast? Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); From 437cbe32f8f25c83e1aa9a08385736f3b88d1eb7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 1 Aug 2014 15:13:31 -0700 Subject: [PATCH 05/14] Update use of dataTypes, fix some python tests, address review comments. --- python/pyspark/sql.py | 16 ++- .../apache/spark/sql/UdfRegistration.scala | 4 +- .../sql/api/java/FunctionRegistration.scala | 4 +- .../spark/sql/execution/pythonUdfs.scala | 21 ++- .../spark/sql/api/java/JavaAPISuite.java | 1 - .../org/apache/spark/sql/PythonUDFSuite.scala | 128 +++++++++--------- .../apache/spark/sql/hive/HiveContext.scala | 1 + 7 files changed, 92 insertions(+), 83 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 55e09f71e626e..d8d645693b350 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -525,14 +525,22 @@ def _ssql_ctx(self): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext - def registerFunction(self, name, f, returnType = "string"): - def func(split, iterator): return imap(f, iterator) - command = (func, self._sc.serializer, self._sc.serializer) + def registerFunction(self, name, f, returnType=StringType()): + func = lambda _, it: imap(lambda x: f(*x), it) + command = (func, + BatchedSerializer(PickleSerializer(), 1024), + BatchedSerializer(PickleSerializer(), 1024)) env = MapConverter().convert(self._sc.environment, self._sc._gateway._gateway_client) includes = ListConverter().convert(self._sc._python_includes, self._sc._gateway._gateway_client) - self._ssql_ctx.registerPython(name, bytearray(CloudPickleSerializer().dumps(command)), env, includes, self._sc.pythonExec, self._sc._javaAccumulator, returnType) + self._ssql_ctx.registerPython(name, + bytearray(CloudPickleSerializer().dumps(command)), + env, + includes, + self._sc.pythonExec, + self._sc._javaAccumulator, + str(returnType)) def inferSchema(self, rdd): """Infer and apply a schema to an RDD of L{dict}s. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 5169fbe104fa8..0e87d8f2ca0f1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -49,6 +49,7 @@ protected[sql] trait UdfRegistration { | envVars: $envVars | pythonIncludes: $pythonIncludes | pythonExec: $pythonExec + | dataType: $dataType """.stripMargin) @@ -59,8 +60,7 @@ protected[sql] trait UdfRegistration { pythonIncludes, pythonExec, accumulator, - org.apache.spark.sql.catalyst.types.StringType, - // FIXME: HiveMetastoreTypes.toDataType(dataType), + parseDataType(dataType), e) functionRegistry.registerFunction(name, builder) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala index 4c6645919338f..b12a76d826702 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.api.java import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} -import org.apache.spark.sql.api.java.types.DataType import org.apache.spark.sql.types.util.DataTypeConversions._ /** @@ -37,7 +36,8 @@ private[java] trait FunctionRegistration { val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") println(s""" - |def registerFunction(name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { + |def registerFunction( + | name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { | val scalaType = asScalaDataType(dataType) | sqlContext.functionRegistry.registerFunction( | name, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 6820fc14bbe3e..b92091b560b1c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -81,6 +81,8 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { // Rewrite the child that has the input required for the UDF val newChildren = l.children.map { child => + // Check to make sure that the UDF can be evaluated with only the input of this child. + // Other cases are disallowed as they are ambiguous or would require a cartisian product. if (udf.references.subsetOf(child.outputSet)) { evaluation = EvaluatePython(udf, child) evaluation @@ -132,16 +134,11 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: val parent = childResults.mapPartitions { iter => val pickle = new Pickler val currentRow = newMutableProjection(udf.children, child.output)() - iter.map { inputRow => - val toBePickled = currentRow(inputRow) - log.debug(s"toBePickled: $toBePickled") - if(children.length == 1) { - pickle.dumps(toBePickled.toArray) - } else { - pickle.dumps(Array(toBePickled.toArray)) - } + iter.grouped(1000).map { inputRows => + val toBePickled = inputRows.map(currentRow(_).toArray).toArray + pickle.dumps(toBePickled) } - }.asInstanceOf[RDD[Any]] + } val pyRDD = new PythonRDD( parent, @@ -155,9 +152,8 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: ).mapPartitions { iter => val pickle = new Unpickler iter.flatMap { pickedResult => - val res = pickle.loads(pickedResult) - log.debug(s"pickleOutput: $res") - res.asInstanceOf[java.util.ArrayList[Any]] + val unpickledBatch = pickle.loads(pickedResult) + unpickledBatch.asInstanceOf[java.util.ArrayList[Any]] } }.mapPartitions { iter => val row = new GenericMutableRow(1) @@ -166,7 +162,6 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: case StringType => result.toString case other => result } - log.debug(s"resultRow: $row") row: Row } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index eff6ff951bab2..c530f706c6cef 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -28,7 +28,6 @@ import org.junit.runner.RunWith; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.api.java.types.DataType; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala index e7ecf40e952e2..df024a5345ec0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala @@ -30,42 +30,45 @@ class PythonUDFSuite extends QueryTest { /* sqlCtx.registerFunction("test", lambda (x): "test:" + x) */ registerPython( "test", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, - 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, - 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, - 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, - 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, - 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, - 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, - 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, - 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, - 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, - 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, + Array[Byte](-128, 2, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, + 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, + 113, 0, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, + 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, + 113, 1, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 2, 40, 75, 2, 75, 2, 75, 3, 75, 19, + 85, 22, 116, 0, 0, -121, 0, 0, 102, 1, 0, 100, 1, 0, -122, 0, 0, 124, 1, 0, -125, 2, 0, 83, + 113, 3, 78, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 19, 85, 10, -120, 0, 0, 124, 0, 0, -116, 0, + 0, 83, 113, 4, 78, -123, 113, 5, 41, 85, 1, 120, 113, 6, -123, 113, 7, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, - 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, - 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, - 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, - 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, - 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, 124, 0, 0, 23, 83, 113, 27, 78, - 85, 5, 116, 101, 115, 116, 58, 113, 28, -122, 113, 29, 41, 85, 1, 120, 113, 30, -123, 113, 31, - 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 32, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, - 113, 33, 75, 1, 85, 0, 113, 34, 41, 41, 116, 113, 35, 82, 113, 36, 75, 0, 125, 113, 37, -121, - 113, 38, 82, 113, 39, 125, 113, 40, 78, 93, 113, 41, 125, 113, 42, 116, 82, 97, 125, 113, 43, - 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, - 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, - 10, 113, 44, 41, -127, 113, 45, 125, 113, 46, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, - 101, 113, 47, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 48, 99, - 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, - 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 49, - 41, -127, 113, 50, 125, 113, 51, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, - 95, 115, 116, 114, 105, 110, 103, 115, 113, 52, -119, 115, 98, 117, 98, 104, 45, -121, 113, - 53, 46), + 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 8, 85, 8, 60, 108, 97, 109, 98, 100, 97, + 62, 113, 9, 77, 17, 2, 85, 0, 113, 10, 85, 1, 102, 113, 11, -123, 113, 12, 41, 116, 113, 13, + 82, 113, 14, -122, 113, 15, 85, 4, 105, 109, 97, 112, 113, 16, -123, 113, 17, 85, 1, 95, 113, + 18, 85, 2, 105, 116, 113, 19, -122, 113, 20, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, + 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, + 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, + 113, 108, 46, 112, 121, 113, 21, 104, 9, 77, 17, 2, 104, 10, 104, 11, -123, 113, 22, 41, 116, + 113, 23, 82, 113, 24, 75, 1, 125, 113, 25, -121, 113, 26, 82, 113, 27, 125, 113, 28, 104, 16, + 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 29, 115, 78, + 93, 113, 30, 104, 0, 40, 104, 1, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, + 124, 0, 0, 23, 83, 113, 31, 78, 85, 5, 116, 101, 115, 116, 58, 113, 32, -122, 113, 33, 41, + 104, 6, -123, 113, 34, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 35, 104, 9, 75, 1, 85, 0, + 113, 36, 41, 41, 116, 113, 37, 82, 113, 38, 75, 0, 125, 113, 39, -121, 113, 40, 82, 113, 41, + 125, 113, 42, 78, 93, 113, 43, 125, 113, 44, 116, 82, 97, 125, 113, 45, 116, 82, 99, 112, 121, + 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, + 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 46, 41, -127, + 113, 47, 125, 113, 48, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 49, 77, 0, 4, + 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 50, 99, 112, 121, 115, 112, 97, + 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, + 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 51, 41, -127, 113, 52, 125, 113, + 53, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, + 103, 115, 113, 54, -119, 115, 98, 117, 98, 104, 46, 41, -127, 113, 55, 125, 113, 56, 40, 104, + 49, 77, 0, 4, 104, 50, 104, 51, 41, -127, 113, 57, 125, 113, 58, 104, 54, -119, 115, 98, 117, + 98, -121, 113, 59, 46), new java.util.HashMap[String, String](), new java.util.LinkedList[String](), "python", null, - "string" + "StringType" ) /** @@ -115,7 +118,7 @@ class PythonUDFSuite extends QueryTest { new java.util.LinkedList[String](), "python", null, - "int" + "IntegerType" ) /** @@ -123,42 +126,45 @@ class PythonUDFSuite extends QueryTest { */ registerPython( "len", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, - 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, - 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, - 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, - 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, - 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, - 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, - 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, - 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, - 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, - 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, + Array[Byte](-128, 2, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, + 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, + 113, 0, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, + 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, + 113, 1, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 2, 40, 75, 2, 75, 2, 75, 3, 75, 19, + 85, 22, 116, 0, 0, -121, 0, 0, 102, 1, 0, 100, 1, 0, -122, 0, 0, 124, 1, 0, -125, 2, 0, 83, + 113, 3, 78, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 19, 85, 10, -120, 0, 0, 124, 0, 0, -116, 0, + 0, 83, 113, 4, 78, -123, 113, 5, 41, 85, 1, 120, 113, 6, -123, 113, 7, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, - 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, - 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, - 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, - 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, - 5, 104, 6, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, 124, 0, 0, -125, 1, 0, 83, 113, - 27, 78, -123, 113, 28, 85, 3, 108, 101, 110, 113, 29, -123, 113, 30, 85, 3, 115, 116, 114, - 113, 31, -123, 113, 32, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 33, 85, 8, 60, 108, 97, - 109, 98, 100, 97, 62, 113, 34, 75, 1, 85, 0, 113, 35, 41, 41, 116, 113, 36, 82, 113, 37, 75, - 0, 125, 113, 38, -121, 113, 39, 82, 113, 40, 125, 113, 41, 78, 93, 113, 42, 125, 113, 43, 116, - 82, 97, 125, 113, 44, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, - 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, - 108, 105, 122, 101, 114, 10, 113, 45, 41, -127, 113, 46, 125, 113, 47, 40, 85, 9, 98, 97, 116, - 99, 104, 83, 105, 122, 101, 113, 48, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, - 101, 114, 113, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, - 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, - 114, 10, 113, 50, 41, -127, 113, 51, 125, 113, 52, 85, 19, 95, 111, 110, 108, 121, 95, 119, - 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 53, -119, 115, 98, 117, 98, - 104, 46, -121, 113, 54, 46), + 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 8, 85, 8, 60, 108, 97, 109, 98, 100, 97, + 62, 113, 9, 77, 17, 2, 85, 0, 113, 10, 85, 1, 102, 113, 11, -123, 113, 12, 41, 116, 113, 13, + 82, 113, 14, -122, 113, 15, 85, 4, 105, 109, 97, 112, 113, 16, -123, 113, 17, 85, 1, 95, 113, + 18, 85, 2, 105, 116, 113, 19, -122, 113, 20, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, + 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, + 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, + 113, 108, 46, 112, 121, 113, 21, 104, 9, 77, 17, 2, 104, 10, 104, 11, -123, 113, 22, 41, 116, + 113, 23, 82, 113, 24, 75, 1, 125, 113, 25, -121, 113, 26, 82, 113, 27, 125, 113, 28, 104, 16, + 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 29, 115, 78, + 93, 113, 30, 104, 0, 40, 104, 1, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, + 124, 0, 0, -125, 1, 0, 83, 113, 31, 78, -123, 113, 32, 85, 3, 108, 101, 110, 113, 33, -123, + 113, 34, 85, 3, 115, 116, 114, 113, 35, -123, 113, 36, 85, 7, 60, 115, 116, 100, 105, 110, 62, + 113, 37, 104, 9, 75, 1, 85, 0, 113, 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, + 41, -121, 113, 42, 82, 113, 43, 125, 113, 44, 78, 93, 113, 45, 125, 113, 46, 116, 82, 97, 125, + 113, 47, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, + 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, + 101, 114, 10, 113, 48, 41, -127, 113, 49, 125, 113, 50, 40, 85, 9, 98, 97, 116, 99, 104, 83, + 105, 122, 101, 113, 51, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, + 113, 52, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, + 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, + 113, 53, 41, -127, 113, 54, 125, 113, 55, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, + 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 56, -119, 115, 98, 117, 98, 104, 48, 41, + -127, 113, 57, 125, 113, 58, 40, 104, 51, 77, 0, 4, 104, 52, 104, 53, 41, -127, 113, 59, 125, + 113, 60, 104, 56, -119, 115, 98, 117, 98, -121, 113, 61, 46), new java.util.HashMap[String, String](), new java.util.LinkedList[String](), "python", null, - "int" + "IntegerType" ) test("Single argument UDF") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 70dde87eb67e2..0b0716b4ca44d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -156,6 +156,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } + // Note that HiveUDFs will be overridden by functions registered in this context. override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry From e54fb4524f6f093ade469dd8371bf1834f186548 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 1 Aug 2014 15:51:10 -0700 Subject: [PATCH 06/14] Docs and tests. --- python/pyspark/sql.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d8d645693b350..74b7fedc918e6 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -526,6 +526,17 @@ def _ssql_ctx(self): return self._scala_SQLContext def registerFunction(self, name, f, returnType=StringType()): + """Registers a lambda function as a UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not given it default to a string and conversion will automatically + be done. For any other return type, the produced object must match the specified type. + + + sqlCtx.registerFunction("stringLengthPy", lambda x: len(x)) + rdd = sc.parallelize(xrange(10)).map(lambda x: {"s": x}) + sqlCtx.inferSchema(rdd).registerAsTable("stringData") + """ func = lambda _, it: imap(lambda x: f(*x), it) command = (func, BatchedSerializer(PickleSerializer(), 1024), From 795fd151d56705a69cfcb750bb744a26c2074a68 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 1 Aug 2014 15:51:27 -0700 Subject: [PATCH 07/14] Try to avoid capturing SQLContext. --- .../scala/org/apache/spark/sql/UdfRegistration.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 0e87d8f2ca0f1..98061ecd0a691 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -40,8 +40,8 @@ protected[sql] trait UdfRegistration { pythonIncludes: JList[String], pythonExec: String, accumulator: Accumulator[JList[Array[Byte]]], - dataType: String): Unit = { - logger.debug( + stringDataType: String): Unit = { + log.debug( s""" | Registering new PythonUDF: | name: $name @@ -49,10 +49,12 @@ protected[sql] trait UdfRegistration { | envVars: $envVars | pythonIncludes: $pythonIncludes | pythonExec: $pythonExec - | dataType: $dataType + | dataType: $stringDataType """.stripMargin) + val dataType = parseDataType(stringDataType) + def builder(e: Seq[Expression]) = PythonUDF(name, command, @@ -60,7 +62,7 @@ protected[sql] trait UdfRegistration { pythonIncludes, pythonExec, accumulator, - parseDataType(dataType), + dataType, e) functionRegistry.registerFunction(name, builder) From 7a83101d479d507b2dac14edba6c5a1a8735a137 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 1 Aug 2014 15:51:41 -0700 Subject: [PATCH 08/14] Drop toString --- .../src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala index df024a5345ec0..2045c1f457484 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala @@ -171,7 +171,7 @@ class PythonUDFSuite extends QueryTest { checkAnswer( sql("SELECT test(value) FROM testData"), testData.select('value).map(r => - new GenericRow(Array[Any]("test:" + r(0).toString))).collect().toSeq) + new GenericRow(Array[Any]("test:" + r(0)))).collect().toSeq) } test("Multiple argument UDF") { From 6a3689074590cc84d7b3e041772917f183d80574 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 1 Aug 2014 15:57:29 -0700 Subject: [PATCH 09/14] Switch logging so that SQLContext can be serializable. --- .../scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/UdfRegistration.scala | 3 ++- .../org/apache/spark/sql/hive/HiveContext.scala | 2 +- .../scala/org/apache/spark/sql/hive/TestHive.scala | 14 +++++++------- 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 66dbd9e4345e5..53b914cbd6b7a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -48,7 +48,7 @@ import org.apache.spark.SparkContext */ @AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) - extends Logging + extends org.apache.spark.Logging with SQLConf with ExpressionConversions with UdfRegistration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 98061ecd0a691..88b26be24f378 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -56,7 +56,8 @@ protected[sql] trait UdfRegistration { val dataType = parseDataType(stringDataType) def builder(e: Seq[Expression]) = - PythonUDF(name, + PythonUDF( + name, command, envVars, pythonIncludes, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 0b0716b4ca44d..3a5a62942f87d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -212,7 +212,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } catch { case e: Exception => - logger.error( + log.error( s""" |====================== |HIVE FAILURE OUTPUT diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c50e8c4b5c5d3..8d4a0492476ad 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -148,7 +148,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { describedTables ++ logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) - logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + log.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. analyzer(logical) @@ -273,7 +273,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. loadedTables += name - logger.info(s"Loading test table $name") + log.info(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -297,8 +297,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { def reset() { try { // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach { logger => - logger.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) + org.apache.log4j.LogManager.getCurrentLoggers.foreach { log => + log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } // It is important that we RESET first as broken hooks that might have been set could break @@ -312,7 +312,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadedTables.clear() catalog.client.getAllTables("default").foreach { t => - logger.debug(s"Deleting table $t") + log.debug(s"Deleting table $t") val table = catalog.client.getTable("default", t) catalog.client.getIndexes("default", t, 255).foreach { index => @@ -325,7 +325,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logger.debug(s"Dropping Database: $db") + log.debug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -347,7 +347,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") + log.error(s"FATAL ERROR: Failed to reset TestDB state. $e") // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. From 8135c484a5192fcd37349ef7d70879a443b193f5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 00:42:55 -0700 Subject: [PATCH 10/14] Move UDF unit tests to pyspark. --- python/pyspark/sql.py | 13 +- .../org/apache/spark/sql/PythonUDFSuite.scala | 205 ------------------ 2 files changed, 9 insertions(+), 209 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index df9336d5f14c1..e7c35ac1ffe02 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -943,10 +943,15 @@ def registerFunction(self, name, f, returnType=StringType()): When the return type is not given it default to a string and conversion will automatically be done. For any other return type, the produced object must match the specified type. - - sqlCtx.registerFunction("stringLengthPy", lambda x: len(x)) - rdd = sc.parallelize(xrange(10)).map(lambda x: {"s": x}) - sqlCtx.inferSchema(rdd).registerAsTable("stringData") + >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + [Row(c0=u'4')] + >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + >>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + >>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + [Row(c0=5)] """ func = lambda _, it: imap(lambda x: f(*x), it) command = (func, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala deleted file mode 100644 index 2045c1f457484..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/PythonUDFSuite.scala +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql - -import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.test.TestSQLContext - -class PythonUDFSuite extends QueryTest { - import TestSQLContext._ - - val testData = sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") - - /* sqlCtx.registerFunction("test", lambda (x): "test:" + x) */ - registerPython( - "test", - Array[Byte](-128, 2, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, - 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, - 113, 0, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, - 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, - 113, 1, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 2, 40, 75, 2, 75, 2, 75, 3, 75, 19, - 85, 22, 116, 0, 0, -121, 0, 0, 102, 1, 0, 100, 1, 0, -122, 0, 0, 124, 1, 0, -125, 2, 0, 83, - 113, 3, 78, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 19, 85, 10, -120, 0, 0, 124, 0, 0, -116, 0, - 0, 83, 113, 4, 78, -123, 113, 5, 41, 85, 1, 120, 113, 6, -123, 113, 7, 85, 53, 47, 85, 115, - 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, - 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, - 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 8, 85, 8, 60, 108, 97, 109, 98, 100, 97, - 62, 113, 9, 77, 17, 2, 85, 0, 113, 10, 85, 1, 102, 113, 11, -123, 113, 12, 41, 116, 113, 13, - 82, 113, 14, -122, 113, 15, 85, 4, 105, 109, 97, 112, 113, 16, -123, 113, 17, 85, 1, 95, 113, - 18, 85, 2, 105, 116, 113, 19, -122, 113, 20, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, - 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, - 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, - 113, 108, 46, 112, 121, 113, 21, 104, 9, 77, 17, 2, 104, 10, 104, 11, -123, 113, 22, 41, 116, - 113, 23, 82, 113, 24, 75, 1, 125, 113, 25, -121, 113, 26, 82, 113, 27, 125, 113, 28, 104, 16, - 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 29, 115, 78, - 93, 113, 30, 104, 0, 40, 104, 1, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 8, 100, 1, 0, - 124, 0, 0, 23, 83, 113, 31, 78, 85, 5, 116, 101, 115, 116, 58, 113, 32, -122, 113, 33, 41, - 104, 6, -123, 113, 34, 85, 7, 60, 115, 116, 100, 105, 110, 62, 113, 35, 104, 9, 75, 1, 85, 0, - 113, 36, 41, 41, 116, 113, 37, 82, 113, 38, 75, 0, 125, 113, 39, -121, 113, 40, 82, 113, 41, - 125, 113, 42, 78, 93, 113, 43, 125, 113, 44, 116, 82, 97, 125, 113, 45, 116, 82, 99, 112, 121, - 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 66, 97, - 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 46, 41, -127, - 113, 47, 125, 113, 48, 40, 85, 9, 98, 97, 116, 99, 104, 83, 105, 122, 101, 113, 49, 77, 0, 4, - 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 113, 50, 99, 112, 121, 115, 112, 97, - 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, 115, 10, 80, 105, 99, 107, 108, - 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, 113, 51, 41, -127, 113, 52, 125, 113, - 53, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, 116, 101, 95, 115, 116, 114, 105, 110, - 103, 115, 113, 54, -119, 115, 98, 117, 98, 104, 46, 41, -127, 113, 55, 125, 113, 56, 40, 104, - 49, 77, 0, 4, 104, 50, 104, 51, 41, -127, 113, 57, 125, 113, 58, 104, 54, -119, 115, 98, 117, - 98, -121, 113, 59, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "StringType" - ) - - /** - * import re - * sqlCtx.registerFunction("countMatches", - * lambda (pattern, string): re.subn(pattern, '', string)[1], returnType = "int") - */ - registerPython( - "countMatches", - Array[Byte](-128, 2, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, - 105, 99, 107, 108, 101, 10, 95, 109, 111, 100, 117, 108, 101, 115, 95, 116, 111, 95, 109, 97, - 105, 110, 10, 113, 0, 93, 113, 1, 85, 11, 112, 121, 115, 112, 97, 114, 107, 46, 115, 113, 108, - 113, 2, 97, -123, 113, 3, 82, 49, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, - 100, 112, 105, 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, - 111, 110, 10, 113, 4, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, - 112, 105, 99, 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, - 110, 99, 10, 113, 5, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 6, 40, 75, 2, 75, 2, - 75, 3, 75, 19, 85, 13, 116, 0, 0, -120, 0, 0, 124, 1, 0, -125, 2, 0, 83, 113, 7, 78, -123, - 113, 8, 85, 4, 105, 109, 97, 112, 113, 9, -123, 113, 10, 85, 5, 115, 112, 108, 105, 116, 113, - 11, 85, 8, 105, 116, 101, 114, 97, 116, 111, 114, 113, 12, -122, 113, 13, 85, 53, 47, 85, 115, - 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, - 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, - 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 14, 85, 4, 102, 117, 110, 99, 113, 15, 75, - 81, 85, 0, 113, 16, 85, 1, 102, 113, 17, -123, 113, 18, 41, 116, 113, 19, 82, 113, 20, 75, 1, - 125, 113, 21, -121, 113, 22, 82, 113, 23, 125, 113, 24, 104, 9, 99, 105, 116, 101, 114, 116, - 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 25, 115, 78, 93, 113, 26, 104, 4, 40, 104, - 5, 104, 6, 40, 75, 1, 75, 3, 75, 4, 75, 67, 85, 35, 124, 0, 0, 92, 2, 0, 125, 1, 0, 125, 2, 0, - 116, 0, 0, 106, 1, 0, 124, 1, 0, 100, 1, 0, 124, 2, 0, -125, 3, 0, 100, 2, 0, 25, 83, 113, 27, - 78, 104, 16, 75, 1, -121, 113, 28, 85, 2, 114, 101, 113, 29, 85, 4, 115, 117, 98, 110, 113, - 30, -122, 113, 31, 85, 2, 46, 48, 113, 32, 85, 7, 112, 97, 116, 116, 101, 114, 110, 113, 33, - 85, 6, 115, 116, 114, 105, 110, 103, 113, 34, -121, 113, 35, 85, 7, 60, 115, 116, 100, 105, - 110, 62, 113, 36, 85, 8, 60, 108, 97, 109, 98, 100, 97, 62, 113, 37, 75, 1, 85, 2, 3, 0, 113, - 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, 41, -121, 113, 42, 82, 113, 43, 125, - 113, 44, 104, 29, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, - 99, 107, 108, 101, 10, 115, 117, 98, 105, 109, 112, 111, 114, 116, 10, 113, 45, 85, 2, 114, - 101, 113, 46, -123, 113, 47, 82, 113, 48, 115, 78, 93, 113, 49, 125, 113, 50, 116, 82, 97, - 125, 113, 51, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, - 105, 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, - 122, 101, 114, 10, 113, 52, 41, -127, 113, 53, 125, 113, 54, 40, 85, 9, 98, 97, 116, 99, 104, - 83, 105, 122, 101, 113, 55, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, - 113, 56, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, - 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, - 113, 57, 41, -127, 113, 58, 125, 113, 59, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, - 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 60, -119, 115, 98, 117, 98, 104, 53, - -121, 113, 61, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "IntegerType" - ) - - /** - * sqlCtx.registerFunction("len", lambda str: len(str), returnType = "int") - */ - registerPython( - "len", - Array[Byte](-128, 2, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, - 99, 107, 108, 101, 10, 95, 102, 105, 108, 108, 95, 102, 117, 110, 99, 116, 105, 111, 110, 10, - 113, 0, 40, 99, 112, 121, 115, 112, 97, 114, 107, 46, 99, 108, 111, 117, 100, 112, 105, 99, - 107, 108, 101, 10, 95, 109, 97, 107, 101, 95, 115, 107, 101, 108, 95, 102, 117, 110, 99, 10, - 113, 1, 99, 110, 101, 119, 10, 99, 111, 100, 101, 10, 113, 2, 40, 75, 2, 75, 2, 75, 3, 75, 19, - 85, 22, 116, 0, 0, -121, 0, 0, 102, 1, 0, 100, 1, 0, -122, 0, 0, 124, 1, 0, -125, 2, 0, 83, - 113, 3, 78, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 19, 85, 10, -120, 0, 0, 124, 0, 0, -116, 0, - 0, 83, 113, 4, 78, -123, 113, 5, 41, 85, 1, 120, 113, 6, -123, 113, 7, 85, 53, 47, 85, 115, - 101, 114, 115, 47, 109, 97, 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, - 99, 101, 47, 115, 112, 97, 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, - 97, 114, 107, 47, 115, 113, 108, 46, 112, 121, 113, 8, 85, 8, 60, 108, 97, 109, 98, 100, 97, - 62, 113, 9, 77, 17, 2, 85, 0, 113, 10, 85, 1, 102, 113, 11, -123, 113, 12, 41, 116, 113, 13, - 82, 113, 14, -122, 113, 15, 85, 4, 105, 109, 97, 112, 113, 16, -123, 113, 17, 85, 1, 95, 113, - 18, 85, 2, 105, 116, 113, 19, -122, 113, 20, 85, 53, 47, 85, 115, 101, 114, 115, 47, 109, 97, - 114, 109, 98, 114, 117, 115, 47, 119, 111, 114, 107, 115, 112, 97, 99, 101, 47, 115, 112, 97, - 114, 107, 47, 112, 121, 116, 104, 111, 110, 47, 112, 121, 115, 112, 97, 114, 107, 47, 115, - 113, 108, 46, 112, 121, 113, 21, 104, 9, 77, 17, 2, 104, 10, 104, 11, -123, 113, 22, 41, 116, - 113, 23, 82, 113, 24, 75, 1, 125, 113, 25, -121, 113, 26, 82, 113, 27, 125, 113, 28, 104, 16, - 99, 105, 116, 101, 114, 116, 111, 111, 108, 115, 10, 105, 109, 97, 112, 10, 113, 29, 115, 78, - 93, 113, 30, 104, 0, 40, 104, 1, 104, 2, 40, 75, 1, 75, 1, 75, 2, 75, 67, 85, 10, 116, 0, 0, - 124, 0, 0, -125, 1, 0, 83, 113, 31, 78, -123, 113, 32, 85, 3, 108, 101, 110, 113, 33, -123, - 113, 34, 85, 3, 115, 116, 114, 113, 35, -123, 113, 36, 85, 7, 60, 115, 116, 100, 105, 110, 62, - 113, 37, 104, 9, 75, 1, 85, 0, 113, 38, 41, 41, 116, 113, 39, 82, 113, 40, 75, 0, 125, 113, - 41, -121, 113, 42, 82, 113, 43, 125, 113, 44, 78, 93, 113, 45, 125, 113, 46, 116, 82, 97, 125, - 113, 47, 116, 82, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, - 122, 101, 114, 115, 10, 66, 97, 116, 99, 104, 101, 100, 83, 101, 114, 105, 97, 108, 105, 122, - 101, 114, 10, 113, 48, 41, -127, 113, 49, 125, 113, 50, 40, 85, 9, 98, 97, 116, 99, 104, 83, - 105, 122, 101, 113, 51, 77, 0, 4, 85, 10, 115, 101, 114, 105, 97, 108, 105, 122, 101, 114, - 113, 52, 99, 112, 121, 115, 112, 97, 114, 107, 46, 115, 101, 114, 105, 97, 108, 105, 122, 101, - 114, 115, 10, 80, 105, 99, 107, 108, 101, 83, 101, 114, 105, 97, 108, 105, 122, 101, 114, 10, - 113, 53, 41, -127, 113, 54, 125, 113, 55, 85, 19, 95, 111, 110, 108, 121, 95, 119, 114, 105, - 116, 101, 95, 115, 116, 114, 105, 110, 103, 115, 113, 56, -119, 115, 98, 117, 98, 104, 48, 41, - -127, 113, 57, 125, 113, 58, 40, 104, 51, 77, 0, 4, 104, 52, 104, 53, 41, -127, 113, 59, 125, - 113, 60, 104, 56, -119, 115, 98, 117, 98, -121, 113, 61, 46), - new java.util.HashMap[String, String](), - new java.util.LinkedList[String](), - "python", - null, - "IntegerType" - ) - - test("Single argument UDF") { - checkAnswer( - sql("SELECT test(value) FROM testData"), - testData.select('value).map(r => - new GenericRow(Array[Any]("test:" + r(0)))).collect().toSeq) - } - - test("Multiple argument UDF") { - checkAnswer( - sql("SELECT countMatches('1', value) FROM testData"), - testData.select('value).map(r => - new GenericRow(Array[Any]( - "1".r.findAllMatchIn(r(0).toString).length))).collect().toSeq) - } - - test("UDF in WHERE") { - checkAnswer( - sql("SELECT value FROM testData WHERE countMatches('1', value) >= 1"), - testData - .select('value) - .filter(r => "1".r.findAllMatchIn(r(0).toString).length >= 1) - .collect().toSeq) - } - - test("Integer Return Type") { - sparkContext.parallelize(1 to 20).map(i => new StringData("*" * i)).registerAsTable("stars") - checkAnswer( - sql("SELECT len(str) FROM stars"), - table("stars").map(r => - new GenericRow(Array[Any](r.getString(0).length))).collect().toSeq - ) - - } -} - -case class StringData(str: String) \ No newline at end of file From d14dac8b35c6b57dc03b8e8ac2bd441cd60be69b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 13:24:38 -0700 Subject: [PATCH 11/14] Fix last line of autogened java files. --- sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java | 1 - sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java | 1 - .../org/apache/spark/sql/api/java/FunctionRegistration.scala | 2 +- 23 files changed, 1 insertion(+), 23 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java index dc07b2203bc23..ef959e35e1027 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -30,4 +30,3 @@ public interface UDF1 extends Serializable { public R call(T1 t1) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java index 7358654e1c748..96ab3a96c3d5e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -30,4 +30,3 @@ public interface UDF10 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java index 51f66b67151d0..58ae8edd6d817 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -30,4 +30,3 @@ public interface UDF11 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java index daedb250b58c7..d9da0f6eddd94 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -30,4 +30,3 @@ public interface UDF12 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java index 1ef3fc2d8df1f..095fc1a8076b5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -30,4 +30,3 @@ public interface UDF13 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java index f1a872c96863e..eb27eaa180086 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -30,4 +30,3 @@ public interface UDF14 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java index 91ae68eb5b317..1fbcff56332b6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -30,4 +30,3 @@ public interface UDF15 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java index deb45150b6c78..1133561787a69 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -30,4 +30,3 @@ public interface UDF16 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java index 14fd59a31181c..dfae7922c9b63 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -30,4 +30,3 @@ public interface UDF17 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java index 364c3e87b862d..e9d1c6d52d4ea 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -30,4 +30,3 @@ public interface UDF18 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java index 1193d45124b2e..46b9d2d3c9457 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -30,4 +30,3 @@ public interface UDF19 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java index fc77c3cf8249d..cd3fde8da419e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -30,4 +30,3 @@ public interface UDF2 extends Serializable { public R call(T1 t1, T2 t2) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java index 89d2600870dab..113d3d26be4a7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -30,4 +30,3 @@ public interface UDF20 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java index 6a084b929a37e..74118f2cf8da7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -30,4 +30,3 @@ public interface UDF21 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java index 4d9ee070e77b4..0e7cc40be45ec 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -30,4 +30,3 @@ public interface UDF22 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java index eb90d5b152f48..6a880f16be47a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -30,4 +30,3 @@ public interface UDF3 extends Serializable { public R call(T1 t1, T2 t2, T3 t3) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java index cf731e334f2ec..fcad2febb18e6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -30,4 +30,3 @@ public interface UDF4 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java index 5501f5e31cbd7..ce0cef43a2144 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -30,4 +30,3 @@ public interface UDF5 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java index b42e9aa2639c8..f56b806684e61 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -30,4 +30,3 @@ public interface UDF6 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java index 9d5d571bb413d..25bd6d3241bd4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -30,4 +30,3 @@ public interface UDF7 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java index 36caade21dc22..a3b7ac5f94ce7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -30,4 +30,3 @@ public interface UDF8 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java index f934d93914daa..205e72a1522fc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -30,4 +30,3 @@ public interface UDF9 extends Serializable { public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; } - \ No newline at end of file diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala index b12a76d826702..c0e855c60fb16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala @@ -86,7 +86,7 @@ private[java] trait FunctionRegistration { |public interface UDF$i<$typeArgs, R> extends Serializable { | public R call($args) throws Exception; |} - """.stripMargin + |""".stripMargin stringToFile(new File(directory, s"UDF$i.java"), contents) } From 005d6842d115870105bd8a2d2aa6f7bbd6e89aae Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 14:11:05 -0700 Subject: [PATCH 12/14] Fix naming and formatting. --- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/UdfRegistration.scala | 7 +++-- .../spark/sql/api/java/JavaSQLContext.scala | 2 +- ...gistration.scala => UDFRegistration.scala} | 26 +++---------------- .../spark/sql/api/java/JavaAPISuite.java | 2 +- 5 files changed, 9 insertions(+), 30 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/api/java/{FunctionRegistration.scala => UDFRegistration.scala} (99%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 33280173fb7f5..5b26f54945eee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -51,7 +51,7 @@ class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging with SQLConf with ExpressionConversions - with UdfRegistration + with UDFRegistration with Serializable { self => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 88b26be24f378..0b48e9e659faa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -26,11 +26,10 @@ import org.apache.spark.sql.execution.PythonUDF import scala.reflect.runtime.universe.{TypeTag, typeTag} - /** - * Functions for registering UDFs in a context. + * Functions for registering scala lambda functions as UDFs in a SQLContext. */ -protected[sql] trait UdfRegistration { +protected[sql] trait UDFRegistration { self: SQLContext => private[spark] def registerPython( @@ -194,4 +193,4 @@ protected[sql] trait UdfRegistration { functionRegistry.registerFunction(name, builder) } // scalastyle:on -} \ No newline at end of file +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 0882c4ce48ef5..ae45193ed15d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(val sqlContext: SQLContext) extends FunctionRegistration { +class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala rename to sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala index c0e855c60fb16..158f26e3d445f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/FunctionRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.types.util.DataTypeConversions._ * of varying airities with minimal boilerplate for our users, we generate classes and functions * for each airity up to 22. The code for this generation can be found in comments in this trait. */ -private[java] trait FunctionRegistration { +private[java] trait UDFRegistration { self: JavaSQLContext => /* The following functions and required interfaces are generated with these code fragments: @@ -101,7 +101,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -109,7 +108,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -117,7 +115,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -125,7 +122,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -133,7 +129,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -141,7 +136,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -149,7 +143,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -157,7 +150,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -165,7 +157,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -173,7 +164,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -181,7 +171,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -189,7 +178,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -197,7 +185,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -205,7 +192,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -213,7 +199,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -221,7 +206,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -229,7 +213,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -237,7 +220,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -245,7 +227,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -253,7 +234,6 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( @@ -261,12 +241,12 @@ private[java] trait FunctionRegistration { (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } - def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { val scalaType = asScalaDataType(dataType) sqlContext.functionRegistry.registerFunction( name, (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) } + // scalastyle:on -} \ No newline at end of file +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java index c530f706c6cef..a9a11285def54 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -87,4 +87,4 @@ public Integer call(String str1, String str2) throws Exception { Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); assert(result.getInt(0) == 9); } -} \ No newline at end of file +} From 747c05ea707f54faf46a22f1809ac05b6686e44c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 14:27:53 -0700 Subject: [PATCH 13/14] Add some scala UDF tests. --- .../scala/org/apache/spark/sql/UDFSuite.scala | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala 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 new file mode 100644 index 0000000000000..0be8bd72192e7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +class UDFSuite extends QueryTest { + + test("Simple UDF") { + registerFunction("strLenScala", (_: String).length) + assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) + } + + test("TwoArgument UDF") { + registerFunction("strLenScala", (_: String).length + (_:Int)) + assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) + } +} \ No newline at end of file From 9eda0fef2e500afe5919013b0afd0eb64ff0c213 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 14:47:40 -0700 Subject: [PATCH 14/14] newline --- sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0be8bd72192e7..76aa9b0081d7e 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 @@ -33,4 +33,4 @@ class UDFSuite extends QueryTest { registerFunction("strLenScala", (_: String).length + (_:Int)) assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) } -} \ No newline at end of file +}