Skip to content

Commit dc7b387

Browse files
dilipbiswalyhuai
authored andcommitted
[SPARK-12558][SQL] AnalysisException when multiple functions applied in GROUP BY clause
cloud-fan Can you please take a look ? In this case, we are failing during check analysis while validating the aggregation expression. I have added a semanticEquals for HiveGenericUDF to fix this. Please let me know if this is the right way to address this issue. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #10520 from dilipbiswal/spark-12558.
1 parent f14922c commit dc7b387

File tree

2 files changed

+30
-0
lines changed

2 files changed

+30
-0
lines changed

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,11 +26,13 @@ import scala.reflect.ClassTag
2626

2727
import com.esotericsoftware.kryo.Kryo
2828
import com.esotericsoftware.kryo.io.{Input, Output}
29+
import com.google.common.base.Objects
2930
import org.apache.avro.Schema
3031
import org.apache.hadoop.conf.Configuration
3132
import org.apache.hadoop.fs.Path
3233
import org.apache.hadoop.hive.ql.exec.{UDF, Utilities}
3334
import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc}
35+
import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro
3436
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils
3537
import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils}
3638
import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector
@@ -45,6 +47,7 @@ private[hive] object HiveShim {
4547
// scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs)
4648
val UNLIMITED_DECIMAL_PRECISION = 38
4749
val UNLIMITED_DECIMAL_SCALE = 18
50+
val HIVE_GENERIC_UDF_MACRO_CLS = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro"
4851

4952
/*
5053
* This function in hive-0.13 become private, but we have to do this to walkaround hive bug
@@ -123,6 +126,26 @@ private[hive] object HiveShim {
123126
// for Serialization
124127
def this() = this(null)
125128

129+
override def hashCode(): Int = {
130+
if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) {
131+
Objects.hashCode(functionClassName, instance.asInstanceOf[GenericUDFMacro].getBody())
132+
} else {
133+
functionClassName.hashCode()
134+
}
135+
}
136+
137+
override def equals(other: Any): Boolean = other match {
138+
case a: HiveFunctionWrapper if functionClassName == a.functionClassName =>
139+
// In case of udf macro, check to make sure they point to the same underlying UDF
140+
if (functionClassName == HIVE_GENERIC_UDF_MACRO_CLS) {
141+
a.instance.asInstanceOf[GenericUDFMacro].getBody() ==
142+
instance.asInstanceOf[GenericUDFMacro].getBody()
143+
} else {
144+
true
145+
}
146+
case _ => false
147+
}
148+
126149
@transient
127150
def deserializeObjectByKryo[T: ClassTag](
128151
kryo: Kryo,

sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -350,6 +350,13 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils {
350350
sqlContext.dropTempTable("testUDF")
351351
}
352352

353+
test("Hive UDF in group by") {
354+
Seq(Tuple1(1451400761)).toDF("test_date").registerTempTable("tab1")
355+
val count = sql("select date(cast(test_date as timestamp))" +
356+
" from tab1 group by date(cast(test_date as timestamp))").count()
357+
assert(count == 1)
358+
}
359+
353360
test("SPARK-11522 select input_file_name from non-parquet table"){
354361

355362
withTempDir { tempDir =>

0 commit comments

Comments
 (0)