Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -69,49 +69,56 @@ private[sql] class HiveSessionCatalog(
// 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.makeFunctionExpression(name, clazz, input)).getOrElse {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AngersZhuuuu, can you get rid of this unrelated diffs?

Copy link
Contributor Author

@AngersZhuuuu AngersZhuuuu Aug 21, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AngersZhuuuu, can you get rid of this unrelated diffs?

How about current change, it won't change indentation. cc @maropu

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))
udfExpr.get.asInstanceOf[HiveGenericUDTF].elementSchema // Force it to check data types.
Try(super.makeFunctionExpression(name, clazz, input)) match {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AngersZhuuuu Seems we don't need to change getOrElse to match.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@AngersZhuuuu Seems we don't need to change getOrElse to match.

Confused, with getOrElse how can I get the exception

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If so, how about using try instead of Try?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

try is the way as in the guideline. But in general I would like to avoid changing the indentation for unrelated codes and make it difficult to see what's the real diff. Such pattern makes it more difficult to backport and revert. Let's avoid this @AngersZhuuuu next time. The actual diff seems just adding more message in the exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

try is the way as in the guideline. But in general I would like to avoid changing the indentation for unrelated codes and make it difficult to see what's the real diff. Such pattern makes it more difficult to backport and revert. Let's avoid this @AngersZhuuuu next time. The actual diff seems just adding more message in the exception.

Since I always backport pr, I know the point you mentioned, I will notice this more carefully.
Yea, the actual diff is to show more message but since we need to know the exception in return Failure(exception), so we can't use getOrElse()

case Success(value) => value
case Failure(exception) =>
val superError =
s"Make Expression failed in SessionCatalog for function name = ${name}:" +
s" ${exception.getMessage}"
var udfExpr: Option[Expression] = None
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can move the code below to a new method.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can move the code below to a new method.

Done

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(e) =>
val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e"
val errorMsg =
if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
s"$noHandlerMsg\nPlease make sure your function overrides " +
"`public StructObjectInspector initialize(ObjectInspector[] args)`."
} else {
noHandlerMsg
}
val analysisException = new AnalysisException(s"1. $superError\n2. $errorMsg")
analysisException.setStackTrace(e.getStackTrace)
throw analysisException
}
udfExpr.getOrElse {
throw new AnalysisException(
s"1. $superError\n2. No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'")
}
} catch {
case NonFatal(e) =>
val noHandlerMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e"
val errorMsg =
if (classOf[GenericUDTF].isAssignableFrom(clazz)) {
s"$noHandlerMsg\nPlease make sure your function overrides " +
"`public StructObjectInspector initialize(ObjectInspector[] args)`."
} else {
noHandlerMsg
}
val analysisException = new AnalysisException(errorMsg)
analysisException.setStackTrace(e.getStackTrace)
throw analysisException
}
udfExpr.getOrElse {
throw new AnalysisException(s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'")
}
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,6 +161,21 @@ class HiveUDAFSuite extends QueryTest
checkAnswer(sql("select histogram_numeric(a,2) from abc where a=3"), Row(null))
}
}

test("Hive mode use spark udaf should show error") {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's add a JIRA prefix.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's add a JIRA prefix.

Done

val functionName = "longProductSum"
val functionClass = "org.apache.spark.sql.hive.execution.LongProductSum"
withUserDefinedFunction(functionName -> true) {
sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'")
val e1 = intercept[AnalysisException] {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: e1 -> e

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: e1 -> e

Done

sql(s"SELECT $functionName(100)")
}.getMessage
assert(
Seq(s"Invalid number of arguments for function $functionName. Expected: 2; Found: 1;",
"No handler for UDF/UDAF/UDTF 'org.apache.spark.sql.hive.execution.LongProductSum';")
.forall(e1.contains))
}
}
}

/**
Expand Down