-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19439][PYSPARK][SQL] PySpark's registerJavaFunction Should Support UDAFs #17222
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,6 @@ | |
|
|
||
| package org.apache.spark.sql | ||
|
|
||
| import java.io.IOException | ||
| import java.lang.reflect.{ParameterizedType, Type} | ||
|
|
||
| import scala.reflect.runtime.universe.TypeTag | ||
|
|
@@ -456,9 +455,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | |
| .map(_.asInstanceOf[ParameterizedType]) | ||
| .filter(e => e.getRawType.isInstanceOf[Class[_]] && e.getRawType.asInstanceOf[Class[_]].getCanonicalName.startsWith("org.apache.spark.sql.api.java.UDF")) | ||
| if (udfInterfaces.length == 0) { | ||
| throw new IOException(s"UDF class ${className} doesn't implement any UDF interface") | ||
| throw new AnalysisException(s"UDF class ${className} doesn't implement any UDF interface") | ||
| } else if (udfInterfaces.length > 1) { | ||
| throw new IOException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") | ||
| throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class ${className}") | ||
| } else { | ||
| try { | ||
| val udf = clazz.newInstance() | ||
|
|
@@ -491,19 +490,41 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | |
| case 21 => register(name, udf.asInstanceOf[UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) | ||
| case 22 => register(name, udf.asInstanceOf[UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) | ||
| case 23 => register(name, udf.asInstanceOf[UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _]], returnType) | ||
| case n => logError(s"UDF class with ${n} type arguments is not supported ") | ||
| case n => | ||
| throw new AnalysisException(s"UDF class with ${n} type arguments is not supported.") | ||
| } | ||
| } catch { | ||
| case e @ (_: InstantiationException | _: IllegalArgumentException) => | ||
| logError(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") | ||
| throw new AnalysisException(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") | ||
| } | ||
| } | ||
| } catch { | ||
| case e: ClassNotFoundException => logError(s"Can not load class ${className}, please make sure it is on the classpath") | ||
| case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") | ||
| } | ||
|
|
||
| } | ||
|
|
||
| /** | ||
| * Register a Java UDAF class using reflection, for use from pyspark | ||
| * | ||
| * @param name UDAF name | ||
| * @param className fully qualified class name of UDAF | ||
| */ | ||
| private[sql] def registerJavaUDAF(name: String, className: String): Unit = { | ||
|
||
| try { | ||
| val clazz = Utils.classForName(className) | ||
| if (!classOf[UserDefinedAggregateFunction].isAssignableFrom(clazz)) { | ||
| throw new AnalysisException(s"class $className doesn't implement interface UserDefinedAggregateFunction") | ||
| } | ||
| val udaf = clazz.newInstance().asInstanceOf[UserDefinedAggregateFunction] | ||
| register(name, udaf) | ||
| } catch { | ||
| case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") | ||
| case e @ (_: InstantiationException | _: IllegalArgumentException) => | ||
| throw new AnalysisException(s"Can not instantiate class ${className}, please make sure it has public non argument constructor") | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Register a user-defined function with 1 arguments. | ||
| * @since 1.3.0 | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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 test.org.apache.spark.sql; | ||
|
|
||
| import org.apache.spark.sql.Row; | ||
| import org.apache.spark.sql.SparkSession; | ||
| import org.junit.After; | ||
| import org.junit.Assert; | ||
| import org.junit.Before; | ||
| import org.junit.Test; | ||
|
|
||
|
|
||
| public class JavaUDAFSuite { | ||
|
|
||
| private transient SparkSession spark; | ||
|
|
||
| @Before | ||
| public void setUp() { | ||
| spark = SparkSession.builder() | ||
| .master("local[*]") | ||
| .appName("testing") | ||
| .getOrCreate(); | ||
| } | ||
|
|
||
| @After | ||
| public void tearDown() { | ||
| spark.stop(); | ||
| spark = null; | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @Test | ||
| public void udf1Test() { | ||
| spark.range(1, 10).toDF("value").registerTempTable("df"); | ||
| spark.udf().registerJavaUDAF("myDoubleAvg", MyDoubleAvg.class.getName()); | ||
| Row result = spark.sql("SELECT myDoubleAvg(value) as my_avg from df").head(); | ||
| Assert.assertEquals(105.0, result.getDouble(0), 1.0e-6); | ||
| } | ||
|
|
||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,7 +31,7 @@ | |
| import org.apache.spark.sql.expressions.UserDefinedAggregateFunction; | ||
| import static org.apache.spark.sql.functions.*; | ||
| import org.apache.spark.sql.hive.test.TestHive$; | ||
| import org.apache.spark.sql.hive.aggregate.MyDoubleSum; | ||
| import test.org.apache.spark.sql.MyDoubleSum; | ||
|
|
||
| public class JavaDataFrameSuite { | ||
|
||
| private transient SQLContext hc; | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,16 +20,19 @@ package org.apache.spark.sql.hive.execution | |
| import scala.collection.JavaConverters._ | ||
| import scala.util.Random | ||
|
|
||
| import test.org.apache.spark.sql.MyDoubleAvg | ||
| import test.org.apache.spark.sql.MyDoubleSum | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeRow | ||
| import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction} | ||
| import org.apache.spark.sql.functions._ | ||
| import org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum} | ||
| import org.apache.spark.sql.hive.test.TestHiveSingleton | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.test.SQLTestUtils | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
|
|
||
|
||
| class ScalaAggregateFunction(schema: StructType) extends UserDefinedAggregateFunction { | ||
|
|
||
| def inputSchema: StructType = schema | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Missing @SInCE.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@SInCE is needed for private function ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK. I did not notice it.