diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala index b8cced0b80969..e0d9c40123370 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala @@ -123,6 +123,12 @@ private[hive] object HiveShim { // for Serialization def this() = this(null) + override def equals(other: Any): Boolean = other match { + case a: HiveFunctionWrapper => + functionClassName == a.functionClassName + case _ => false + } + @transient def deserializeObjectByKryo[T: ClassTag]( kryo: Kryo, 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 74fcbf72b98fa..b1a6d0ab7df3c 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 @@ -232,20 +232,6 @@ private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, childr udfType != null && udfType.deterministic() } - override def semanticEquals(other: Expression): Boolean = { - val eqClass = other.isInstanceOf[HiveGenericUDF] && - funcWrapper.functionClassName == - other.asInstanceOf[HiveGenericUDF].funcWrapper.functionClassName - - val isEqual = eqClass && - children.length == other.asInstanceOf[HiveGenericUDF].children.length && - children.zip(other.asInstanceOf[HiveGenericUDF].children).forall { - case (e1: Expression, e2: Expression) => e1 semanticEquals e2 - case (i1, i2) => i1 == i2 - } - isEqual - } - @transient private lazy val deferedObjects = argumentInspectors.zip(children).map { case (inspect, child) => new DeferredObjectAdapter(inspect, child.dataType)