-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-12558][SQL][WIP] AnalysisException when multiple functions applied in GROUP BY clause #10520
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 4 commits
c9f2e92
de81c21
7dae8e1
24b32d5
b1e2d13
41f6f02
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 |
|---|---|---|
|
|
@@ -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 | ||
|
|
@@ -123,6 +126,24 @@ private[hive] object HiveShim { | |
| // for Serialization | ||
| def this() = this(null) | ||
|
|
||
| override def hashCode(): Int = { | ||
| if (instance != null && instance.isInstanceOf[GenericUDFMacro]) { | ||
| Objects.hashCode(functionClassName, instance) | ||
| } else { | ||
| functionClassName.hashCode() | ||
| } | ||
|
Contributor
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. 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, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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() | ||
|
Contributor
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. 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
Contributor
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. 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 => | ||
|
|
||
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.
Actually, should we check the class name?
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.
I mean when we check if this function wrapper is for a GenericUDFMacro, should we check the class name?
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.
@yhuai Thanks !! Could users have a class that extends GenericUDFMacro ? Wouldn't we want to handle that case as well ?
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.
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.).
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.
@yhuai Ah.. thanks a lot. I understand now that its not possible for users to extend the macro.I will make the change.