-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-37075][SQL] Move UDAF expression building from sql/catalyst to sql/core #34340
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 |
|---|---|---|
| @@ -0,0 +1,31 @@ | ||
| /* | ||
| * 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.catalyst.catalog | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
|
|
||
| // A builder to create `Expression` from function information. | ||
| trait FunctionExpressionBuilder { | ||
| def makeExpression(name: String, clazz: Class[_], input: Seq[Expression]): Expression | ||
| } | ||
|
|
||
| object DummyFunctionExpressionBuilder extends FunctionExpressionBuilder { | ||
| override def makeExpression(name: String, clazz: Class[_], input: Seq[Expression]): Expression = { | ||
| throw new UnsupportedOperationException | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,9 +17,17 @@ | |
|
|
||
| package org.apache.spark.sql.hive | ||
|
|
||
| import java.lang.reflect.InvocationTargetException | ||
|
|
||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} | ||
| import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} | ||
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.analysis.{Analyzer, ResolveSessionCatalog} | ||
| import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener | ||
| import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} | ||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.execution.SparkPlanner | ||
|
|
@@ -29,9 +37,10 @@ import org.apache.spark.sql.execution.command.CommandCheck | |
| import org.apache.spark.sql.execution.datasources._ | ||
| import org.apache.spark.sql.execution.datasources.v2.TableCapabilityCheck | ||
| import org.apache.spark.sql.execution.streaming.ResolveWriteToStream | ||
| import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper | ||
| import org.apache.spark.sql.hive.client.HiveClient | ||
| import org.apache.spark.sql.hive.execution.PruneHiveTablePartitions | ||
| import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} | ||
| import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState, SparkUDFExpressionBuilder} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
|
|
@@ -64,7 +73,8 @@ class HiveSessionStateBuilder( | |
| tableFunctionRegistry, | ||
| SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), | ||
| sqlParser, | ||
| resourceLoader) | ||
| resourceLoader, | ||
| HiveUDFExpressionBuilder) | ||
| parentState.foreach(_.catalog.copyStateTo(catalog)) | ||
| catalog | ||
| } | ||
|
|
@@ -133,3 +143,68 @@ class HiveSessionResourceLoader( | |
| } | ||
| } | ||
| } | ||
|
|
||
| object HiveUDFExpressionBuilder extends SparkUDFExpressionBuilder { | ||
| override def makeExpression(name: String, clazz: Class[_], input: Seq[Expression]): Expression = { | ||
| // Current thread context classloader may not be the one loaded the class. Need to switch | ||
| // context classloader to initialize instance properly. | ||
| Utils.withContextClassLoader(clazz.getClassLoader) { | ||
| try { | ||
| super.makeExpression(name, clazz, input) | ||
| } catch { | ||
| // If `super.makeFunctionExpression` throw `InvalidUDFClassException`, we construct | ||
| // Hive UDF/UDAF/UDTF with function definition. Otherwise, we just throw it earlier. | ||
| case _: InvalidUDFClassException => | ||
| makeHiveFunctionExpression(name, clazz, input) | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @cloud-fan in prior spark version, it will convert decimal to double when exception occuring, #13930 . At now, some hive udfs which receive double throw UDFArgumentException. The udf extends GenericUDF.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Was it already broken before this PR? The removed code in this PR does not have this handling. Anyway, I'm happy to review the fix if you can create a PR, thanks! |
||
| case NonFatal(e) => throw e | ||
| } | ||
| } | ||
| } | ||
|
|
||
| private def makeHiveFunctionExpression( | ||
| name: String, | ||
| clazz: Class[_], | ||
| input: Seq[Expression]): Expression = { | ||
| var udfExpr: Option[Expression] = None | ||
| try { | ||
| // When we instantiate hive UDF wrapper class, we may throw exception if the input | ||
| // expressions don't satisfy the hive UDF, such as type mismatch, input number | ||
| // mismatch, etc. Here we catch the exception and throw AnalysisException instead. | ||
| if (classOf[UDF].isAssignableFrom(clazz)) { | ||
| udfExpr = Some(HiveSimpleUDF(name, new HiveFunctionWrapper(clazz.getName), input)) | ||
| udfExpr.get.dataType // Force it to check input data types. | ||
| } else if (classOf[GenericUDF].isAssignableFrom(clazz)) { | ||
| udfExpr = Some(HiveGenericUDF(name, new HiveFunctionWrapper(clazz.getName), input)) | ||
| udfExpr.get.dataType // Force it to check input data types. | ||
| } else if (classOf[AbstractGenericUDAFResolver].isAssignableFrom(clazz)) { | ||
| udfExpr = Some(HiveUDAFFunction(name, new HiveFunctionWrapper(clazz.getName), input)) | ||
| udfExpr.get.dataType // Force it to check input data types. | ||
| } else if (classOf[UDAF].isAssignableFrom(clazz)) { | ||
| udfExpr = Some(HiveUDAFFunction( | ||
| name, | ||
| new HiveFunctionWrapper(clazz.getName), | ||
| input, | ||
| isUDAFBridgeRequired = true)) | ||
| udfExpr.get.dataType // Force it to check input data types. | ||
| } else if (classOf[GenericUDTF].isAssignableFrom(clazz)) { | ||
| udfExpr = Some(HiveGenericUDTF(name, new HiveFunctionWrapper(clazz.getName), input)) | ||
| // Force it to check data types. | ||
| udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema | ||
| } | ||
| } catch { | ||
| case NonFatal(exception) => | ||
| val e = exception match { | ||
| case i: InvocationTargetException => i.getCause | ||
| case o => o | ||
| } | ||
| val errorMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e" | ||
| val analysisException = new AnalysisException(errorMsg) | ||
| analysisException.setStackTrace(e.getStackTrace) | ||
| throw analysisException | ||
| } | ||
| udfExpr.getOrElse { | ||
| throw new InvalidUDFClassException( | ||
| s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'") | ||
| } | ||
| } | ||
| } | ||
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.
Should we add a bit more doc about what parameters are, for an interface?
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.
sounds good, @beliefer can you add it in your PR? These 3 parameters were not documented before this PR either.
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