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

feat: Implement ANSI support for UnaryMinus #471

Merged
merged 24 commits into from
Jun 3, 2024

Conversation

vaibhawvipul
Copy link
Contributor

@vaibhawvipul vaibhawvipul commented May 25, 2024

Which issue does this PR close?

Closes #465 .

Rationale for this change

Improves compatibility with spark

What changes are included in this PR?

ANSI support for UnaryMinus by adding input checks when ANSI mode enabled.

How are these changes tested?

scala> val df = Seq(Int.MaxValue, Int.MinValue).toDF("a")
scala> df.write.parquet("/tmp/int.parquet")
scala> spark.read.parquet("/tmp/int.parquet").createTempView("t")

scala> spark.conf.set("spark.sql.ansi.enabled", true)
scala> spark.conf.set("spark.comet.ansi.enabled", true)

scala> spark.conf.set("spark.comet.enabled", true)

scala> spark.sql("select a, -a from t").show

returns the expected error

  • Caused by: org.apache.spark.SparkArithmeticException: [ARITHMETIC_OVERFLOW] integer overflow. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.

@vaibhawvipul
Copy link
Contributor Author

Adding evidence that we have parity with spark with and without ANSI mode.

Comet and Spark outputs with ANSI false -

Screenshot 2024-05-26 at 6 55 47 PM

Comet and Spark outputs with ANSI true -

Spark -
Screenshot 2024-05-26 at 6 56 25 PM

Comet -
Screenshot 2024-05-26 at 6 56 53 PM

@vaibhawvipul vaibhawvipul marked this pull request as ready for review May 26, 2024 13:30
Copy link
Member

@andygrove andygrove left a comment

Choose a reason for hiding this comment

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

This is looking good @vaibhawvipul. I think all that is needed now are unit tests perhaps in CometExpressionSuite to show that this is all working as intended.

I also left some minor feedback.

@vaibhawvipul
Copy link
Contributor Author

@andygrove we have a test case now which shows that we have parity.

Copy link
Member

@andygrove andygrove left a comment

Choose a reason for hiding this comment

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

LGTM pending CI

@vaibhawvipul
Copy link
Contributor Author

I have made all the requested changes, to me it looks ready for CI.

Please let me know. cc @kazuyukitanimura @parthchandra

Copy link
Contributor

@parthchandra parthchandra left a comment

Choose a reason for hiding this comment

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

LGTM

Copy link
Contributor

@kazuyukitanimura kazuyukitanimura left a comment

Choose a reason for hiding this comment

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

Sorry I asked many things, but a few more comments

@vaibhawvipul
Copy link
Contributor Author

Sorry I asked many things, but a few more comments

No problem, it greatly improved my contribution. I learnt a lot and I am happy. This exercise will ensure that my next PR won't have this much back-and-forth :)

@vaibhawvipul
Copy link
Contributor Author

@kazuyukitanimura this is ready for review.

Copy link
Contributor

@kazuyukitanimura kazuyukitanimura left a comment

Choose a reason for hiding this comment

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

This PR went through enough iterations, so I intend to accept this PR.

My comments below can be updated by follow ups.

Comment on lines +1543 to +1552
withTable("t_interval") {
spark.sql("CREATE TABLE t_interval(a STRING) USING PARQUET")
spark.sql("INSERT INTO t_interval VALUES ('INTERVAL 10000000000 YEAR')")
withAnsiMode(enabled = true) {
spark
.sql("SELECT CAST(a AS INTERVAL) AS a FROM t_interval")
.createOrReplaceTempView("t_interval_casted")
checkOverflow("SELECT a, -a FROM t_interval_casted", "interval")
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

It looks this does not hit native code as CAST(a AS INTERVAL) is not supported yet. So it will fall back to Spark and checkOverflow is comparing Spark results on both sides.

Perhaps there is no good way of creating interval now...

Comment on lines +1554 to +1568
withTable("t") {
sql("create table t(a int) using parquet")
sql("insert into t values (-2147483648)")
withAnsiMode(enabled = true) {
checkOverflow("select a, -a from t", "integer")
}
}

withTable("t_float") {
sql("create table t_float(a float) using parquet")
sql("insert into t_float values (3.4128235E38)")
withAnsiMode(enabled = true) {
checkOverflow("select a, -a from t_float", "float")
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

BTW this will not test scalar case unless we do something like

            withSQLConf(
              "spark.sql.optimizer.excludedRules" ->
                "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") {
checkOverflow("select a, -(a) from t_float", "float")

I think the current test creates a single item array.

That said, unless that option is set, it is unlikely to hit the scalar scenario. It is ideal to test scalar cases because user jobs may use that option, but it can be a follow up fix.

Comment on lines +135 to +140
arrow::datatypes::IntervalUnit::DayTime => check_overflow!(
array,
arrow::array::IntervalDayTimeArray,
i64::MIN,
"interval"
),
Copy link
Contributor

Choose a reason for hiding this comment

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

I was expecting to see that testing this fails because DataFusion neg_wrapping breaks the i64 into two i32.

Then I realized there is no good way of testing right now as mentioned above.

Copy link
Member

@andygrove andygrove left a comment

Choose a reason for hiding this comment

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

Thanks again @vaibhawvipul and thank you for the reviews @kazuyukitanimura and @parthchandra

@andygrove andygrove merged commit edd63ef into apache:main Jun 3, 2024
43 checks passed
@vaibhawvipul vaibhawvipul deleted the issue-465 branch June 4, 2024 02:12
@planga82
Copy link
Contributor

planga82 commented Jun 4, 2024

Hi @vaibhawvipul ,

I don't know why, but it seems that a test included in this PR is failing in the main branch in spak 4.0 tests. Any thoughts on this? Thanks in advance!

(- unary negative integer overflow test *** FAILED *** (739 milliseconds)
"[ARITHMETIC_OVERFLOW] short overflow. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. SQLSTATE: 22003" did not contain " caused overflow" (CometExpressionSuite.scala:1563))

@vaibhawvipul
Copy link
Contributor Author

Hi @vaibhawvipul ,

I don't know why, but it seems that a test included in this PR is failing in the main branch in spak 4.0 tests. Any thoughts on this? Thanks in advance!

(- unary negative integer overflow test *** FAILED *** (739 milliseconds) "[ARITHMETIC_OVERFLOW] short overflow. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error. SQLSTATE: 22003" did not contain " caused overflow" (CometExpressionSuite.scala:1563))

Hmm.. I think there was another PR which got merged after mine, that is breaking the tests.
Screenshot 2024-06-04 at 7 19 10 PM

@kazuyukitanimura
Copy link
Contributor

Ops, looks like this is a merge conflict, I will fix it. Sorry for the inconvenience.

@andygrove
Copy link
Member

Ops, looks like this is a merge conflict, I will fix it. Sorry for the inconvenience.

@kazuyukitanimura @vaibhawvipul This is already fixed in main. I fixed it as part of #505

@kazuyukitanimura
Copy link
Contributor

Ah thank you @andygrove cc @planga82

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.

feat: Implement ANSI support for UnaryMinus
5 participants