Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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 @@ -86,14 +86,7 @@ private[sql] class HiveSessionCatalog(
}
} 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 errorMsg = s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}': $e"
val analysisException = new AnalysisException(errorMsg)
analysisException.setStackTrace(e.getStackTrace)
throw analysisException
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -213,10 +213,14 @@ private[hive] case class HiveGenericUDTF(
}

@transient
protected lazy val inputInspectors = children.map(toInspector)
protected lazy val inputInspector = {
val inspectors = children.map(toInspector)
val fields = inspectors.indices.map(index => s"_col$index").asJava
ObjectInspectorFactory.getStandardStructObjectInspector(fields, inspectors.asJava)
}

@transient
protected lazy val outputInspector = function.initialize(inputInspectors.toArray)
protected lazy val outputInspector = function.initialize(inputInspector)

@transient
protected lazy val udtInput = new Array[AnyRef](children.length)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2160,32 +2160,6 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
}
}

test("SPARK-21101 UDTF should override initialize(ObjectInspector[] args)") {
withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) {
sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack1
|AS 'org.apache.spark.sql.hive.execution.UDTFStack'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
""".stripMargin)
val cnt =
sql("SELECT udtf_stack1(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')").count()
assert(cnt === 2)

sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack2
|AS 'org.apache.spark.sql.hive.execution.UDTFStack2'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
""".stripMargin)
val e = intercept[org.apache.spark.sql.AnalysisException] {
sql("SELECT udtf_stack2(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')")
}
assert(
e.getMessage.contains("public StructObjectInspector initialize(ObjectInspector[] args)"))
}
}

test("SPARK-21721: Clear FileSystem deleterOnExit cache if path is successfully removed") {
val table = "test21721"
withTable(table) {
Expand Down Expand Up @@ -2583,6 +2557,30 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi
}
}
}

test("SPARK-32668: HiveGenericUDTF initialize UDTF should use StructObjectInspector method") {
withUserDefinedFunction("udtf_stack1" -> true, "udtf_stack2" -> true) {
sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack1
|AS 'org.apache.spark.sql.hive.execution.UDTFStack'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
""".stripMargin)
sql(
s"""
|CREATE TEMPORARY FUNCTION udtf_stack2
|AS 'org.apache.spark.sql.hive.execution.UDTFStack2'
|USING JAR '${hiveContext.getHiveFile("SPARK-21101-1.0.jar").toURI}'
""".stripMargin)

Seq("udtf_stack1", "udtf_stack2").foreach { udf =>
checkAnswer(
sql(s"SELECT $udf(2, 'A', 10, date '2015-01-01', 'B', 20, date '2016-01-01')"),
Seq(Row("A", 10, Date.valueOf("2015-01-01")),
Row("B", 20, Date.valueOf("2016-01-01"))))
}
}
}
}

@SlowHiveTest
Expand Down