aboutsummaryrefslogtreecommitdiff
path: root/sql/hive
diff options
context:
space:
mode:
authorBenFradet <benjamin.fradet@gmail.com>2015-06-29 15:27:13 -0700
committerMichael Armbrust <michael@databricks.com>2015-06-29 15:27:13 -0700
commit931da5c8ab271ff2ee04419c7e3c6b0012459694 (patch)
tree3ee9c2712c58ab2967315fc8d1f2b53b06002eff /sql/hive
parentc8ae887ef02b8f7e2ad06841719fb12eacf1f7f9 (diff)
downloadspark-931da5c8ab271ff2ee04419c7e3c6b0012459694.tar.gz
spark-931da5c8ab271ff2ee04419c7e3c6b0012459694.tar.bz2
spark-931da5c8ab271ff2ee04419c7e3c6b0012459694.zip
[SPARK-8478] [SQL] Harmonize UDF-related code to use uniformly UDF instead of Udf
Follow-up of #6902 for being coherent between ```Udf``` and ```UDF``` Author: BenFradet <benjamin.fradet@gmail.com> Closes #6920 from BenFradet/SPARK-8478 and squashes the following commits: c500f29 [BenFradet] renamed a few variables in functions to use UDF 8ab0f2d [BenFradet] renamed idUdf to idUDF in SQLQuerySuite 98696c2 [BenFradet] renamed originalUdfs in TestHive to originalUDFs 7738f74 [BenFradet] modified HiveUDFSuite to use only UDF c52608d [BenFradet] renamed HiveUdfSuite to HiveUDFSuite e51b9ac [BenFradet] renamed ExtractPythonUdfs to ExtractPythonUDFs 8c756f1 [BenFradet] renamed Hive UDF related code 2a1ca76 [BenFradet] renamed pythonUdfs to pythonUDFs 261e6fb [BenFradet] renamed ScalaUdf to ScalaUDF
Diffstat (limited to 'sql/hive')
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala4
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala2
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala (renamed from sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala)26
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala4
-rwxr-xr-xsql/hive/src/test/resources/data/files/testUDF/part-00000 (renamed from sql/hive/src/test/resources/data/files/testUdf/part-00000)bin153 -> 153 bytes
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala (renamed from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala)24
6 files changed, 30 insertions, 30 deletions
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 8021f915bb..b91242af2d 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
@@ -42,7 +42,7 @@ import org.apache.spark.sql.SQLConf.SQLConfEntry._
import org.apache.spark.sql.catalyst.ParserDialect
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUdfs, SetCommand}
+import org.apache.spark.sql.execution.{ExecutedCommand, ExtractPythonUDFs, SetCommand}
import org.apache.spark.sql.hive.client._
import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, HiveNativeCommand}
import org.apache.spark.sql.sources.DataSourceStrategy
@@ -381,7 +381,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
catalog.ParquetConversions ::
catalog.CreateTables ::
catalog.PreInsertionCasts ::
- ExtractPythonUdfs ::
+ ExtractPythonUDFs ::
ResolveHiveWindowFunction ::
sources.PreInsertCastAndRename ::
Nil
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 7c4620952b..2de7a99c12 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -1638,7 +1638,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C
sys.error(s"Couldn't find function $functionName"))
val functionClassName = functionInfo.getFunctionClass.getName
- (HiveGenericUdtf(
+ (HiveGenericUDTF(
new HiveFunctionWrapper(functionClassName),
children.map(nodeToExpr)), attributes)
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 4986b1ea9d..d7827d56ca 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
@@ -59,16 +59,16 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
val functionClassName = functionInfo.getFunctionClass.getName
if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveSimpleUdf(new HiveFunctionWrapper(functionClassName), children)
+ HiveSimpleUDF(new HiveFunctionWrapper(functionClassName), children)
} else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdf(new HiveFunctionWrapper(functionClassName), children)
+ HiveGenericUDF(new HiveFunctionWrapper(functionClassName), children)
} else if (
classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdaf(new HiveFunctionWrapper(functionClassName), children)
+ HiveGenericUDAF(new HiveFunctionWrapper(functionClassName), children)
} else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveUdaf(new HiveFunctionWrapper(functionClassName), children)
+ HiveUDAF(new HiveFunctionWrapper(functionClassName), children)
} else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) {
- HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), children)
+ HiveGenericUDTF(new HiveFunctionWrapper(functionClassName), children)
} else {
sys.error(s"No handler for udf ${functionInfo.getFunctionClass}")
}
@@ -79,7 +79,7 @@ private[hive] class HiveFunctionRegistry(underlying: analysis.FunctionRegistry)
throw new UnsupportedOperationException
}
-private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+private[hive] case class HiveSimpleUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
extends Expression with HiveInspectors with Logging {
type UDFType = UDF
@@ -146,7 +146,7 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector)
override def get(): AnyRef = wrap(func(), oi)
}
-private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
+private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, children: Seq[Expression])
extends Expression with HiveInspectors with Logging {
type UDFType = GenericUDF
@@ -413,7 +413,7 @@ private[hive] case class HiveWindowFunction(
new HiveWindowFunction(funcWrapper, pivotResult, isUDAFBridgeRequired, children)
}
-private[hive] case class HiveGenericUdaf(
+private[hive] case class HiveGenericUDAF(
funcWrapper: HiveFunctionWrapper,
children: Seq[Expression]) extends AggregateExpression
with HiveInspectors {
@@ -441,11 +441,11 @@ private[hive] case class HiveGenericUdaf(
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
}
- def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this)
+ def newInstance(): HiveUDAFFunction = new HiveUDAFFunction(funcWrapper, children, this)
}
/** It is used as a wrapper for the hive functions which uses UDAF interface */
-private[hive] case class HiveUdaf(
+private[hive] case class HiveUDAF(
funcWrapper: HiveFunctionWrapper,
children: Seq[Expression]) extends AggregateExpression
with HiveInspectors {
@@ -474,7 +474,7 @@ private[hive] case class HiveUdaf(
s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})"
}
- def newInstance(): HiveUdafFunction = new HiveUdafFunction(funcWrapper, children, this, true)
+ def newInstance(): HiveUDAFFunction = new HiveUDAFFunction(funcWrapper, children, this, true)
}
/**
@@ -488,7 +488,7 @@ private[hive] case class HiveUdaf(
* Operators that require maintaining state in between input rows should instead be implemented as
* user defined aggregations, which have clean semantics even in a partitioned execution.
*/
-private[hive] case class HiveGenericUdtf(
+private[hive] case class HiveGenericUDTF(
funcWrapper: HiveFunctionWrapper,
children: Seq[Expression])
extends Generator with HiveInspectors {
@@ -553,7 +553,7 @@ private[hive] case class HiveGenericUdtf(
}
}
-private[hive] case class HiveUdafFunction(
+private[hive] case class HiveUDAFFunction(
funcWrapper: HiveFunctionWrapper,
exprs: Seq[Expression],
base: AggregateExpression,
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
index ea325cc93c..7978fdacae 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala
@@ -391,7 +391,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
* Records the UDFs present when the server starts, so we can delete ones that are created by
* tests.
*/
- protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames
+ protected val originalUDFs: JavaSet[String] = FunctionRegistry.getFunctionNames
/**
* Resets the test instance by deleting any tables that have been created.
@@ -410,7 +410,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) {
catalog.client.reset()
catalog.unregisterAllTables()
- FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName =>
+ FunctionRegistry.getFunctionNames.filterNot(originalUDFs.contains(_)).foreach { udfName =>
FunctionRegistry.unregisterTemporaryUDF(udfName)
}
diff --git a/sql/hive/src/test/resources/data/files/testUdf/part-00000 b/sql/hive/src/test/resources/data/files/testUDF/part-00000
index 240a5c1a63..240a5c1a63 100755
--- a/sql/hive/src/test/resources/data/files/testUdf/part-00000
+++ b/sql/hive/src/test/resources/data/files/testUDF/part-00000
Binary files differ
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
index ce5985888f..56b0bef1d0 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
@@ -46,7 +46,7 @@ case class ListStringCaseClass(l: Seq[String])
/**
* A test suite for Hive custom UDFs.
*/
-class HiveUdfSuite extends QueryTest {
+class HiveUDFSuite extends QueryTest {
import TestHive.{udf, sql}
import TestHive.implicits._
@@ -73,7 +73,7 @@ class HiveUdfSuite extends QueryTest {
test("hive struct udf") {
sql(
"""
- |CREATE EXTERNAL TABLE hiveUdfTestTable (
+ |CREATE EXTERNAL TABLE hiveUDFTestTable (
| pair STRUCT<id: INT, value: INT>
|)
|PARTITIONED BY (partition STRING)
@@ -82,15 +82,15 @@ class HiveUdfSuite extends QueryTest {
""".
stripMargin.format(classOf[PairSerDe].getName))
- val location = Utils.getSparkClassLoader.getResource("data/files/testUdf").getFile
+ val location = Utils.getSparkClassLoader.getResource("data/files/testUDF").getFile
sql(s"""
- ALTER TABLE hiveUdfTestTable
- ADD IF NOT EXISTS PARTITION(partition='testUdf')
+ ALTER TABLE hiveUDFTestTable
+ ADD IF NOT EXISTS PARTITION(partition='testUDF')
LOCATION '$location'""")
- sql(s"CREATE TEMPORARY FUNCTION testUdf AS '${classOf[PairUdf].getName}'")
- sql("SELECT testUdf(pair) FROM hiveUdfTestTable")
- sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf")
+ sql(s"CREATE TEMPORARY FUNCTION testUDF AS '${classOf[PairUDF].getName}'")
+ sql("SELECT testUDF(pair) FROM hiveUDFTestTable")
+ sql("DROP TEMPORARY FUNCTION IF EXISTS testUDF")
}
test("SPARK-6409 UDAFAverage test") {
@@ -169,11 +169,11 @@ class HiveUdfSuite extends QueryTest {
StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil).toDF()
testData.registerTempTable("stringTable")
- sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'")
+ sql(s"CREATE TEMPORARY FUNCTION testStringStringUDF AS '${classOf[UDFStringString].getName}'")
checkAnswer(
- sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"),
+ sql("SELECT testStringStringUDF(\"hello\", s) FROM stringTable"),
Seq(Row("hello world"), Row("hello goodbye")))
- sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUdf")
+ sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUDF")
TestHive.reset()
}
@@ -244,7 +244,7 @@ class PairSerDe extends AbstractSerDe {
}
}
-class PairUdf extends GenericUDF {
+class PairUDF extends GenericUDF {
override def initialize(p1: Array[ObjectInspector]): ObjectInspector =
ObjectInspectorFactory.getStandardStructObjectInspector(
Seq("id", "value"),