Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 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
21 changes: 21 additions & 0 deletions sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,14 @@ import scala.reflect.ClassTag

import com.esotericsoftware.kryo.Kryo
import com.esotericsoftware.kryo.io.{Input, Output}
import com.google.common.base.Objects

import org.apache.avro.Schema
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.ql.exec.{UDF, Utilities}
import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils}
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector
Expand Down Expand Up @@ -123,6 +126,24 @@ private[hive] object HiveShim {
// for Serialization
def this() = this(null)

override def hashCode(): Int = {
if (instance != null && instance.isInstanceOf[GenericUDFMacro]) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Actually, should we check the class name?

Copy link
Contributor

Choose a reason for hiding this comment

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

I mean when we check if this function wrapper is for a GenericUDFMacro, should we check the class name?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@yhuai Thanks !! Could users have a class that extends GenericUDFMacro ? Wouldn't we want to handle that case as well ?

Copy link
Contributor

Choose a reason for hiding this comment

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

Hive's macro is kind of special. I do not think users can use a class that extends GenericUDFMacro (You can take a look at hive's function registry. Macro registration is handled by registerMacro.).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@yhuai Ah.. thanks a lot. I understand now that its not possible for users to extend the macro.I will make the change.

Objects.hashCode(functionClassName, instance)
} else {
functionClassName.hashCode()
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Seems we need to correct the format?

}

override def equals(other: Any): Boolean = other match {
case a: HiveFunctionWrapper =>
if (a.instance != null && a.instance.isInstanceOf[GenericUDFMacro]) {
functionClassName == a.functionClassName && instance == a.instance
} else {
functionClassName == a.functionClassName
}
case _ => false
}

@transient
def deserializeObjectByKryo[T: ClassTag](
kryo: Kryo,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -350,6 +350,13 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
sqlContext.dropTempTable("testUDF")
}

test("Hive UDF in group by") {
Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1")
val count = sql("select date(cast(test_date as timestamp))" +
" from tab1 group by date(cast(test_date as timestamp))").count()
Copy link
Contributor

Choose a reason for hiding this comment

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

Actually it will be good to use withTempTable, which will automatically drop the temp table. Also, it will be more robust if we create a temp function based on hive's org.apache.hadoop.hive.ql.udf.generic.GenericUDFToDate. So, even if we later have a native function called date, we can still test the hive udf. We can do the test change in a follow-up pr (the pr can re-use the jira number).

Copy link
Contributor

Choose a reason for hiding this comment

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

I just realize it after I merge the PR. Let's have a pr to improve the test. Thanks!

assert(count == 1)
}

test("SPARK-11522 select input_file_name from non-parquet table"){

withTempDir { tempDir =>
Expand Down