Skip to content
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

[SPARK-12558][SQL][WIP] AnalysisException when multiple functions applied in GROUP BY clause #10520

Closed
wants to merge 6 commits into from

Conversation

dilipbiswal
Copy link
Contributor

@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.

@@ -233,6 +233,18 @@ private[hive] case class HiveGenericUDF(funcWrapper: HiveFunctionWrapper, childr
udfType != null && udfType.deterministic()
}

override def semanticEquals(other: Expression): Boolean = {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can you comment why the version in Expression doesn't work? What about it is too strict?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@nongli Thanks a lot for your feedback. In this case , we have two instances of HiveGenericUDF one from the grouping expression and other from aggregation expression. While doing a semantic equality between them , we unwrap the case class and fall through to the last case statement which does equality between two HiveFunctionWrapper(s) and is failing.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@nongli @cloud-fan We could also override equals in HiveFunctionWrapper to just compare the functionClassName. Would this be a better option ? Please let me know.

Copy link
Contributor

Choose a reason for hiding this comment

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

I prefer to override HiveFunctionWrapper.equals, if the HiveFunctionWrapper.instance has no contribution to equality.

@dilipbiswal
Copy link
Contributor Author

@cloud-fan Hi Wenchen, thanks for your comments. I went through the processing. I think its ok to just override equals and compare the udf class names here. The number of arguments, their semantic equality and deterministic property is handled in the semanticEquals method in expression.scala. Please let me know what you think.

@@ -125,6 +125,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
Copy link
Contributor

Choose a reason for hiding this comment

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

cc @yhuai , Does same function name means exactly same function in hive?

Copy link
Contributor

Choose a reason for hiding this comment

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

@dilipbiswal I feel it is general fine to compare the class name (it is what we do in 1.5. In 1.5, instance is not a part of the constructor). We added instance when we added the support of hive's GenericUDFMacro. I guess we can add a check to see if GenericUDFMacro is the class name. If so, we also check instance. Otherwise, we just use the class name. Also, I guess you need to override hashCode as well.

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 Yin. Will make the changes per your suggestion.

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 Have addressed your comments. Can you please take a look to see if it looks ok ?

@yhuai
Copy link
Contributor

yhuai commented Jan 5, 2016

@dilipbiswal
Copy link
Contributor Author

@yhuai Hi Yin, Here is what i had while answering the question from Nong Li. Please let me know if you have more questions that i need to find the answers for.

In this case , we have two instances of HiveGenericUDF one from the grouping expression and other from aggregation expression. While doing a semantic equality between them , we unwrap the case class and fall through to the last case statement which does equality between two HiveFunctionWrapper(s) and is failing.

Let me know what you think.

@yhuai
Copy link
Contributor

yhuai commented Jan 6, 2016

so, two HiveFunctionWrappers equality check failed because they have different instances?

@dilipbiswal
Copy link
Contributor Author

@yhuai Yes Yin.

@yhuai
Copy link
Contributor

yhuai commented Jan 6, 2016

@dilipbiswal Sorry. I do not think I understand the cause. We only set the instance at https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala#L78 (for GenericUDFMacro)? In most of the case, it will be null. Looks like the equality check should not be the problem?

@dilipbiswal
Copy link
Contributor Author

@yhuai Hi Yin, let me double check. As far as i remember, we come to the last case in expression.scala:semanticEqual with two HiveFunctionWrapper and was failing in equal. With a overriden equal it worked. Let me debug this further and get back...

@dilipbiswal
Copy link
Contributor Author

@yhuai Hi Yin, i remember that in this reproduction, the instances were actually having values i.e they were not null. I will double check on this and get back.

@dilipbiswal
Copy link
Contributor Author

@yhuai So the instance is authored through createFunction() while computing returnInspector.

https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala#L198-#L201

So we think we need to cache the instance in this case as its not a simpleUDF. Let me know your thoughts.

@dilipbiswal
Copy link
Contributor Author

@yhuai Hello Yin, did you have any further insights on this ? Is it safe to override equal to ignore the instance while comparing to see if two functions are same ?

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?

@SparkQA
Copy link

SparkQA commented Jan 12, 2016

Test build #2371 has finished for PR 10520 at commit b1e2d13.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@dilipbiswal
Copy link
Contributor Author

@yhuai Hi Yin, Could you please look at the changes when you get a chance. It turns out that we can't compare the classname and instance for equality for GenericUDFMacro as the instances are different. I am instead comparing the underlying function that is held by GenericUDFMacro. Please let me know what you think.

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

test this please

@SparkQA
Copy link

SparkQA commented Jan 13, 2016

Test build #49275 has finished for PR 10520 at commit 41f6f02.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

Can you remove the WIP from the title?

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

Actually, let me change the title while merging it.

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

LGTM. Merging to master and branch 1.6.

@asfgit asfgit closed this in dc7b387 Jan 13, 2016
asfgit pushed a commit that referenced this pull request Jan 13, 2016
…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 <[email protected]>

Closes #10520 from dilipbiswal/spark-12558.

(cherry picked from commit dc7b387)
Signed-off-by: Yin Huai <[email protected]>

Conflicts:
	sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveShim.scala
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!

@dilipbiswal
Copy link
Contributor Author

@yhuai Thank you very much !!

@dilipbiswal
Copy link
Contributor Author

@yhuai I will make the change in a follow up PR re-using the same JIRA.

@yhuai
Copy link
Contributor

yhuai commented Jan 13, 2016

Great. Thank you!

asfgit pushed a commit that referenced this pull request Jan 18, 2016
…plied in GROUP BY clause

Addresses the comments from Yin.
#10520

Author: Dilip Biswal <[email protected]>

Closes #10758 from dilipbiswal/spark-12558-followup.
asfgit pushed a commit that referenced this pull request Jan 18, 2016
…plied in GROUP BY clause

Addresses the comments from Yin.
#10520

Author: Dilip Biswal <[email protected]>

Closes #10758 from dilipbiswal/spark-12558-followup.

(cherry picked from commit db9a860)
Signed-off-by: Yin Huai <[email protected]>

Conflicts:
	sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants