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-32110][SQL] normalize special floating numbers in HyperLogLog++ #30673

Closed
wants to merge 1 commit into from

Conversation

cloud-fan
Copy link
Contributor

What changes were proposed in this pull request?

Currently, Spark treats 0.0 and -0.0 semantically equal, while it still retains the difference between them so that users can see -0.0 when displaying the data set.

The comparison expressions in Spark take care of the special floating numbers and implement the correct semantic. However, Spark doesn't always use these comparison expressions to compare values, and we need to normalize the special floating numbers before comparing them in these places:

  1. GROUP BY
  2. join keys
  3. window partition keys

This PR fixes one more place that compares values without using comparison expressions: HyperLogLog++

Why are the changes needed?

Fix the query result

Does this PR introduce any user-facing change?

Yes, the result of HyperLogLog++ becomes correct now.

How was this patch tested?

a new test case, and a few more test cases that pass before this PR to improve test coverage.

@github-actions github-actions bot added the SQL label Dec 8, 2020
@@ -554,4 +554,94 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(GreaterThan(Literal(Float.NaN), Literal(Float.NaN)), false)
checkEvaluation(GreaterThan(Literal(0.0F), Literal(-0.0F)), false)
}

test("SPARK-32110: compare special double/float values in array") {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

The new tests here pass before this PR. I'm adding them to prove that nested 0.0/-0.0 is fine. CodegenContext.genComp is very conservative and only does the shortcut when both sides are unsafe format and they equal to each other in binary. for 0.0 and -0.0, they do not equal to each other in binary and will fallback to the element-by-element comparison.

evaluateEstimate(hll, buffer, 1);
}

test("SPARK-32110: add NaN") {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This test passes before this PR, as our hash implementation returns the same hash code for all NaN values. I'm adding it just to make the test cases completed.

@cloud-fan
Copy link
Contributor Author

cc @maropu @viirya @dongjoon-hyun

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37034/

@@ -143,6 +143,28 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] {

case _ => throw new IllegalStateException(s"fail to normalize $expr")
}

val FLOAT_NORMALIZER: Any => Any = (input: Any) => {
Copy link
Member

Choose a reason for hiding this comment

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

Not sure, can this just be a def?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is stateless and being a val is more efficient.


val FLOAT_NORMALIZER: Any => Any = (input: Any) => {
val f = input.asInstanceOf[Float]
if (f.isNaN) {
Copy link
Member

Choose a reason for hiding this comment

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

I think this check isn't necessary, as NaN won't equal -0.0f, so it will be returned on line 154 anyway. Or am I missing that there are different NaNs and this is normalizing them too?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is copied from the existing code. NaN is not a single value and we need to make sure all NaN values have the same binary representation in Spark unsafe row.

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37034/

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Kubernetes integration test starting
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37035/

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Kubernetes integration test status success
URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/37035/

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Test build #132434 has finished for PR 30673 at commit 291e652.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Member

@viirya viirya left a comment

Choose a reason for hiding this comment

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

lgtm

@viirya
Copy link
Member

viirya commented Dec 8, 2020

The test failure seems to be flaky tests now.

@viirya
Copy link
Member

viirya commented Dec 8, 2020

retest this please

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

+1, LGTM. Thank you for adding extensive test coverage.
I also checked that test("SPARK-32110: add 0.0 and -0.0") ensures this patch.
Merged to master/3.1/3.0.

The CI failures are known and irrelevant to this one.

dongjoon-hyun pushed a commit that referenced this pull request Dec 8, 2020
### What changes were proposed in this pull request?

Currently, Spark treats 0.0 and -0.0 semantically equal, while it still retains the difference between them so that users can see -0.0 when displaying the data set.

The comparison expressions in Spark take care of the special floating numbers and implement the correct semantic. However, Spark doesn't always use these comparison expressions to compare values, and we need to normalize the special floating numbers before comparing them in these places:
1. GROUP BY
2. join keys
3. window partition keys

This PR fixes one more place that compares values without using comparison expressions: HyperLogLog++

### Why are the changes needed?

Fix the query result

### Does this PR introduce _any_ user-facing change?

Yes, the result of HyperLogLog++ becomes correct now.

### How was this patch tested?

a new test case, and a few more test cases that pass before this PR to improve test coverage.

Closes #30673 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 6fd2345)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
dongjoon-hyun pushed a commit that referenced this pull request Dec 8, 2020
### What changes were proposed in this pull request?

Currently, Spark treats 0.0 and -0.0 semantically equal, while it still retains the difference between them so that users can see -0.0 when displaying the data set.

The comparison expressions in Spark take care of the special floating numbers and implement the correct semantic. However, Spark doesn't always use these comparison expressions to compare values, and we need to normalize the special floating numbers before comparing them in these places:
1. GROUP BY
2. join keys
3. window partition keys

This PR fixes one more place that compares values without using comparison expressions: HyperLogLog++

### Why are the changes needed?

Fix the query result

### Does this PR introduce _any_ user-facing change?

Yes, the result of HyperLogLog++ becomes correct now.

### How was this patch tested?

a new test case, and a few more test cases that pass before this PR to improve test coverage.

Closes #30673 from cloud-fan/bug.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
(cherry picked from commit 6fd2345)
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
@dongjoon-hyun
Copy link
Member

Thank you, @cloud-fan , @srowen , @viirya .

@SparkQA
Copy link

SparkQA commented Dec 8, 2020

Test build #132447 has finished for PR 30673 at commit 291e652.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@maropu
Copy link
Member

maropu commented Dec 9, 2020

late lgtm.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
Development

Successfully merging this pull request may close these issues.

6 participants