[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354163883
 
 

 ##
 File path: sql/core/src/test/resources/sql-tests/results/datetime.sql.out
 ##
 @@ -100,64 +100,294 @@ struct
+struct
 -- !query 11 output
-2001-10-05
+2019-01-01
 
 
 -- !query 12
-select 7 + date '2001-09-28'
+select timestamp '2019-01-01\t'
 -- !query 12 schema
-struct
+struct
 -- !query 12 output
-2001-10-05
+2019-01-01 00:00:00
 
 
 -- !query 13
-select date '2001-10-01' - 7
+select timestamp'2011-11-11 11:11:11' + interval '2' day
 -- !query 13 schema
-struct
+struct
 
 Review comment:
   OK it's the existing behavior too, we can revisit it later.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354162370
 
 

 ##
 File path: sql/core/src/test/resources/sql-tests/results/datetime.sql.out
 ##
 @@ -100,64 +100,294 @@ struct
+struct
 -- !query 11 output
-2001-10-05
+2019-01-01
 
 
 -- !query 12
-select 7 + date '2001-09-28'
+select timestamp '2019-01-01\t'
 -- !query 12 schema
-struct
+struct
 -- !query 12 output
-2001-10-05
+2019-01-01 00:00:00
 
 
 -- !query 13
-select date '2001-10-01' - 7
+select timestamp'2011-11-11 11:11:11' + interval '2' day
 -- !query 13 schema
-struct
+struct
 -- !query 13 output
-2001-09-24
+2011-11-13 11:11:11
 
 
 -- !query 14
-select date '2001-10-01' - date '2001-09-28'
+select timestamp'2011-11-11 11:11:11' - interval '2' day
 -- !query 14 schema
-struct
+struct
 -- !query 14 output
-3 days
+2011-11-09 11:11:11
 
 
 -- !query 15
-select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
+select date'2011-11-11 11:11:11' + interval '2' second
 -- !query 15 schema
-struct
+struct
 -- !query 15 output
-2078 hours 48 minutes 47.654322 seconds
+2011-11-11
 
 
 -- !query 16
-select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
+select date'2011-11-11 11:11:11' - interval '2' second
 -- !query 16 schema
-struct
+struct
 -- !query 16 output
--2078 hours -48 minutes -47.654322 seconds
+2011-11-10
 
 
 -- !query 17
-select date '2019-01-01\t'
+select '2011-11-11' - interval '2' day
 -- !query 17 schema
-struct
+struct
 
 Review comment:
   OK it's the existing behavior. We can revisit it later.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354159014
 
 

 ##
 File path: sql/core/src/test/resources/sql-tests/results/datetime.sql.out
 ##
 @@ -100,64 +100,294 @@ struct
+struct
 -- !query 11 output
-2001-10-05
+2019-01-01
 
 
 -- !query 12
-select 7 + date '2001-09-28'
+select timestamp '2019-01-01\t'
 -- !query 12 schema
-struct
+struct
 -- !query 12 output
-2001-10-05
+2019-01-01 00:00:00
 
 
 -- !query 13
-select date '2001-10-01' - 7
+select timestamp'2011-11-11 11:11:11' + interval '2' day
 -- !query 13 schema
-struct
+struct
 -- !query 13 output
-2001-09-24
+2011-11-13 11:11:11
 
 
 -- !query 14
-select date '2001-10-01' - date '2001-09-28'
+select timestamp'2011-11-11 11:11:11' - interval '2' day
 -- !query 14 schema
-struct
+struct
 -- !query 14 output
-3 days
+2011-11-09 11:11:11
 
 
 -- !query 15
-select date'2020-01-01' - timestamp'2019-10-06 10:11:12.345678'
+select date'2011-11-11 11:11:11' + interval '2' second
 -- !query 15 schema
-struct
+struct
 -- !query 15 output
-2078 hours 48 minutes 47.654322 seconds
+2011-11-11
 
 
 -- !query 16
-select timestamp'2019-10-06 10:11:12.345678' - date'2020-01-01'
+select date'2011-11-11 11:11:11' - interval '2' second
 -- !query 16 schema
-struct
+struct
 -- !query 16 output
--2078 hours -48 minutes -47.654322 seconds
+2011-11-10
 
 
 -- !query 17
-select date '2019-01-01\t'
+select '2011-11-11' - interval '2' day
 -- !query 17 schema
-struct
+struct
 
 Review comment:
   it's super weird that this returns string. What was the behavior before?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354158478
 
 

 ##
 File path: sql/core/src/test/resources/sql-tests/results/datetime.sql.out
 ##
 @@ -100,64 +100,294 @@ struct
+struct
 -- !query 11 output
-2001-10-05
+2019-01-01
 
 
 -- !query 12
-select 7 + date '2001-09-28'
+select timestamp '2019-01-01\t'
 -- !query 12 schema
-struct
+struct
 -- !query 12 output
-2001-10-05
+2019-01-01 00:00:00
 
 
 -- !query 13
-select date '2001-10-01' - 7
+select timestamp'2011-11-11 11:11:11' + interval '2' day
 -- !query 13 schema
-struct
+struct
 
 Review comment:
   can we avoid adding cast if not necessary?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354157506
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
 ##
 @@ -23,9 +23,9 @@ import 
org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
  * Test various parser errors.
  */
 class ErrorParserSuite extends AnalysisTest {
-  import CatalystSqlParser._
   import org.apache.spark.sql.catalyst.dsl.expressions._
   import org.apache.spark.sql.catalyst.dsl.plans._
+  import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
 
 Review comment:
   unnecessary change


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-05 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r354157559
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
 ##
 @@ -31,9 +31,9 @@ import org.apache.spark.sql.types.IntegerType
  * There is also SparkSqlParserSuite in sql/core module for parser rules 
defined in sql/core module.
  */
 class PlanParserSuite extends AnalysisTest {
-  import CatalystSqlParser._
   import org.apache.spark.sql.catalyst.dsl.expressions._
   import org.apache.spark.sql.catalyst.dsl.plans._
+  import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
 
 Review comment:
   ditto


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353581578
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ##
 @@ -185,16 +186,17 @@ case class DateAdd(startDate: Expression, days: 
Expression)
   """,
   since = "1.5.0")
 case class DateSub(startDate: Expression, days: Expression)
-  extends BinaryExpression with ImplicitCastInputTypes {
+  extends BinaryExpression with ExpectsInputTypes {
   override def left: Expression = startDate
   override def right: Expression = days
 
-  override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType)
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType))
 
   override def dataType: DataType = DateType
 
   override def nullSafeEval(start: Any, d: Any): Any = {
-start.asInstanceOf[Int] - d.asInstanceOf[Int]
+start.asInstanceOf[Int] - d.asInstanceOf[Number].intValue()
 
 Review comment:
   can we add some UT in `DateExpressionsSuite` to make sure byte/short works?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353550165
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala
 ##
 @@ -151,17 +151,18 @@ case class CurrentBatchTimestamp(
   """,
   since = "1.5.0")
 case class DateAdd(startDate: Expression, days: Expression)
-  extends BinaryExpression with ImplicitCastInputTypes {
+  extends BinaryExpression with ExpectsInputTypes {
 
   override def left: Expression = startDate
   override def right: Expression = days
 
-  override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType)
+  override def inputTypes: Seq[AbstractDataType] =
+Seq(DateType, TypeCollection(IntegerType, ShortType, ByteType))
 
   override def dataType: DataType = DateType
 
   override def nullSafeEval(start: Any, d: Any): Any = {
-start.asInstanceOf[Int] + d.asInstanceOf[Int]
+start.asInstanceOf[Int] + d.toString.toInt
 
 Review comment:
   this is inefficient. How about `d.asInstanceOf[Number].intValue`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353291365
 
 

 ##
 File path: 
sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
 ##
 @@ -1041,37 +1033,33 @@ cannot resolve '(CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP) - CAST(1 AS DECIMAL(
 -- !query 125
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t
 -- !query 125 schema
-struct<>
+struct
 -- !query 125 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(3,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' (date and 
decimal(3,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 126
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t
 -- !query 126 schema
-struct<>
+struct
 -- !query 126 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(5,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' (date and 
decimal(5,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 127
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t
 -- !query 127 schema
-struct<>
+struct
 -- !query 127 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(10,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' (date and 
decimal(10,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 128
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t
 
 Review comment:
   Let's replace `with ImplicitCastInputTypes` with `with ExpectsInputTypes` in 
`DateAdd/DateSubb`, and add a type coercion rule to cast byte/short to int for 
them


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353268590
 
 

 ##
 File path: 
sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalPrecision.sql.out
 ##
 @@ -1041,37 +1033,33 @@ cannot resolve '(CAST('2017-12-11 09:30:00.0' AS 
TIMESTAMP) - CAST(1 AS DECIMAL(
 -- !query 125
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(3, 0)) FROM t
 -- !query 125 schema
-struct<>
+struct
 -- !query 125 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(3,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(3,0)))' (date and 
decimal(3,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 126
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(5, 0)) FROM t
 -- !query 126 schema
-struct<>
+struct
 -- !query 126 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(5,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(5,0)))' (date and 
decimal(5,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 127
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(10, 0)) FROM t
 -- !query 127 schema
-struct<>
+struct
 -- !query 127 output
-org.apache.spark.sql.AnalysisException
-cannot resolve '(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS 
DECIMAL(10,0)))' due to data type mismatch: differing types in 
'(CAST('2017-12-11 09:30:00' AS DATE) - CAST(1 AS DECIMAL(10,0)))' (date and 
decimal(10,0)).; line 1 pos 7
+2017-12-10
 
 
 -- !query 128
 SELECT cast('2017-12-11 09:30:00' as date) - cast(1 as decimal(20, 0)) FROM t
 
 Review comment:
   This looks weird. Does pgsql support it?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353267201
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
 ##
 @@ -56,8 +56,10 @@ trait AnalysisTest extends PlanTest {
   expectedPlan: LogicalPlan,
   caseSensitive: Boolean = true): Unit = {
 val analyzer = getAnalyzer(caseSensitive)
-val actualPlan = analyzer.executeAndCheck(inputPlan, new 
QueryPlanningTracker)
-comparePlans(actualPlan, expectedPlan)
+val tracker = new QueryPlanningTracker
+val actualPlan = analyzer.executeAndCheck(inputPlan, tracker)
+val actualExpectedPlan = analyzer.executeAndCheck(expectedPlan, tracker)
 
 Review comment:
   do we still need this change?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353240891
 
 

 ##
 File path: 
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
 ##
 @@ -209,20 +209,22 @@ class ExpressionParserSuite extends AnalysisTest {
 
   test("binary arithmetic expressions") {
 // Simple operations
-assertEqual("a * b", 'a * 'b)
-assertEqual("a / b", 'a / 'b)
+assertEqual("a * b", UnresolvedMultiply('a, 'b))
 
 Review comment:
   we can update the `assertEqual` method to resolve `UnresolvedAdd` etc. 
manually.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353221092
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
 ##
 @@ -68,10 +68,10 @@ package object dsl {
 def unary_! : Predicate = Not(expr)
 def unary_~ : Expression = BitwiseNot(expr)
 
-def + (other: Expression): Expression = Add(expr, other)
-def - (other: Expression): Expression = Subtract(expr, other)
-def * (other: Expression): Expression = Multiply(expr, other)
-def / (other: Expression): Expression = Divide(expr, other)
+def + (other: Expression): Expression = UnresolvedAdd(expr, other)
 
 Review comment:
   do we have to change the DSL? mostly `+` means `Add`, we can write 
`UnresolvedAdd` directly when necessary.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353219424
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,65 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. if both side are interval, turns it to [[Add]];
+   * 2. else if one side is interval, turns it to [[TimeAdd]];
+   * 3. else if one side is date, turns it to [[DateAdd]] ;
+   * 4. else turns it to [[Add]].
+   *
+   * For [[UnresolvedSubtract]]:
+   * 1. if both side are interval, turns it to [[Subtract]];
+   * 2. else if the right side is an interval, turns it to [[TimeSub]];
+   * 3. else if one side is timestamp, turns it to [[SubtractTimestamps]];
+   * 4. else if the right side is date, turns it to 
[[DateDiff]]/[[SubtractDates]];
+   * 5. else if the left side is date, turns it to [[DateSub]];
+   * 6. else turns it to [[Subtract]].
+   *
+   * For [[UnresolvedMultiply]]:
+   * 1. If one side is interval, turns it to [[MultiplyInterval]];
+   * 2. otherwise, turns it to [[Multiply]].
+   *
+   * For [[UnresolvedDivide]]:
+   * 1. If the left side is interval, turns it to [[DivideInterval]];
+   * 2. otherwise, turns it to [[Divide]].
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case u @ UnresolvedAdd(l, r) if u.childrenResolved => (l.dataType, 
r.dataType) match {
+  case (CalendarIntervalType, CalendarIntervalType) => Add(l, r)
+  case (_, CalendarIntervalType) => Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, _) => Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
+  case (_, DateType) => DateAdd(r, l)
+  case _ => Add(l, r)
+}
+case u @ UnresolvedSubtract(l, r) if u.childrenResolved => 
(l.dataType, r.dataType) match {
+  case (CalendarIntervalType, CalendarIntervalType) => Subtract(l, r)
+  case (_, CalendarIntervalType) => Cast(TimeSub(l, r), l.dataType)
+  case (TimestampType, _) => SubtractTimestamps(l, r)
+  case (_, TimestampType) => SubtractTimestamps(Cast(l, 
TimestampType), r)
 
 Review comment:
   do we need the cast here?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353218003
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.expressions.objects._
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.catalyst.rules.{Rule, _}
 
 Review comment:
   Why this change?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353115036
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
+   * [[TimeAdd]];
+   * 2. else if one side is date, turns it to [[DateAdd]] ;
+   * 3. else turns it to [[Add]].
+   *
+   * For [[UnresolvedSubtract]]:
+   * 1. If the left side is timestamp/date/string and the right side is an 
interval, turns it to
+   * [[TimeSub]];
+   * 2. else if one side is timestamp and the other side is date/timestamp, 
turns it to
+   * [[SubtractTimestamps]];
+   * 3. else if both side are dates, turns it to 
[[DateDiff]]/[[SubtractDates]];
+   * 4. else if the left side is date, turns it to [[DateSub]];
+   * 5. else turns it to [[Subtract]].
+   *
+   * For [[UnresolvedMultiply]]:
+   * 1. If one side is interval and the other side is numeric, turns it to 
[[MultiplyInterval]];
+   * 2. otherwise, turns it to [[Multiply]].
+   *
+   * For [[UnresolvedDivide]]:
+   * 1. If the left side is interval and the right side is numeric, turns it to
+   * [[DivideInterval]];
+   * 2. otherwise, turns it to [[Divide]].
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case u @ UnresolvedAdd(l, r) if u.childrenResolved => (l.dataType, 
r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
 
 Review comment:
   according to the above discussion, I think it's clearer to do
   ```
   case (CalendarIntervalType, CalendarIntervalType) => Add
   case (_, CalendarIntervalType) => TimeAdd
   case (CalendarIntervalType, _) => TimeAdd
   case (_, DateType) => DateAdd
   case (DateType, _) => DateAdd
   case _ => Add
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353051750
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   type coercion rule will cast date/string to timestamp but not other types, 
it matches the expected behavior, right?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-03 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353043904
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   so what's wrong here? pgsql fails and spark fails as well.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353027110
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   for error reporting, I don't think that there is a perfect solution. We can 
say that interval should be int to make the operation legal, or int should be 
timestamp.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353026693
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   do we support interval + int?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r353005949
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   then how about `if one side is interval and the other side is not interval`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352998689
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
 
 Review comment:
   it's better to reduce the coupling between analyzer rule and type coercion 
rule. I think here we should turn into `TimeAdd` if one side is interval, and 
type coercion rule will cast date/string to timestamp for `TimeAdd`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352998814
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,68 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * For [[UnresolvedAdd]]:
+   * 1. If one side is timestamp/date/string and the other side is interval, 
turns it to
+   * [[TimeAdd]];
+   * 2. else if one side is date, turns it to [[DateAdd]] ;
+   * 3. else turns it to [[Add]].
+   *
+   * For [[UnresolvedSubtract]]:
+   * 1. If the left side is timestamp/date/string and the right side is an 
interval, turns it to
 
 Review comment:
   ditto


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352583163
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
 
 Review comment:
   I think we need to carefully design the rule so that it's easy to reason 
about. How about
   ```
   For `UnresolvedAdd`:
   1. if one side is interval, turn it into `TimeAdd`
   2. if one side is date, turn it into `DateAdd`
   3. otherwise, turn it into `Add`
   
   For `UnresolvedSubtract`:
   1. if the right side is interval, turn it into `TimeSub`
   2. if one side is timestamp, turn in into `SubtractTimestamps`
   3. if one side is date, turn it into `DateDiff`/`SubtractDates`
   4. otherwise, turn it into `Subtract`
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352569684
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
 
 Review comment:
   This part is really messy. @maropu  AFAIK there is a proposal to update the 
type coercion rules to follow SQL standard, is it still active?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352569684
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
 
 Review comment:
   This part is really messy. @marope AFAIK there is a proposal to update the 
type coercion rules to follow SQL standard, is it still active?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352481450
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
+  case (_, DateType) => DateAdd(r, l)
+  case (_, _) => Add(l, r)
+}
+case UnresolvedSubtract(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeSub(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeSub(r, l), r.dataType)
+  case (DateType | NullType, DateType) => if 
(conf.usePostgreSQLDialect) {
+DateDiff(l, r)
+  } else {
+SubtractDates(l, r)
+  }
+  case (TimestampType, TimestampType | DateType | NullType) => 
SubtractTimestamps(l, r)
+  case (DateType | NullType, TimestampType) => 
SubtractTimestamps(Cast(l, TimestampType), r)
+  case (DateType, _) => DateSub(l, r)
+  case (_, _) => Subtract(l, r)
+}
+case UnresolvedMultiply(l, r) => (l.dataType, r.dataType) match {
+  case (CalendarIntervalType, _: NumericType | NullType) => 
MultiplyInterval(l, r)
 
 Review comment:
   ditto


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352481317
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
+  case (_, DateType) => DateAdd(r, l)
+  case (_, _) => Add(l, r)
+}
+case UnresolvedSubtract(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeSub(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeSub(r, l), r.dataType)
+  case (DateType | NullType, DateType) => if 
(conf.usePostgreSQLDialect) {
+DateDiff(l, r)
+  } else {
+SubtractDates(l, r)
+  }
+  case (TimestampType, TimestampType | DateType | NullType) => 
SubtractTimestamps(l, r)
+  case (DateType | NullType, TimestampType) => 
SubtractTimestamps(Cast(l, TimestampType), r)
 
 Review comment:
   ditto


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352481278
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
+  case (_, DateType) => DateAdd(r, l)
+  case (_, _) => Add(l, r)
+}
+case UnresolvedSubtract(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeSub(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeSub(r, l), r.dataType)
+  case (DateType | NullType, DateType) => if 
(conf.usePostgreSQLDialect) {
+DateDiff(l, r)
+  } else {
+SubtractDates(l, r)
+  }
+  case (TimestampType, TimestampType | DateType | NullType) => 
SubtractTimestamps(l, r)
 
 Review comment:
   ditto


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-12-02 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r352480283
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -246,6 +247,54 @@ class Analyzer(
   CleanupAliases)
   )
 
+  /**
+   * 1. Turns Add/Subtract of DateType/TimestampType/StringType and 
CalendarIntervalType
+   *to TimeAdd/TimeSub.
+   * 2. Turns Add/Subtract of TimestampType/DateType/IntegerType
+   *and TimestampType/IntegerType/DateType to 
DateAdd/DateSub/SubtractDates and
+   *to SubtractTimestamps.
+   * 3. Turns Multiply/Divide of CalendarIntervalType and NumericType
+   *to MultiplyInterval/DivideInterval
+   */
+  case class ResolveBinaryArithmetic(conf: SQLConf) extends Rule[LogicalPlan] {
+override def apply(plan: LogicalPlan): LogicalPlan = 
plan.resolveOperatorsUp {
+  case p: LogicalPlan => p.transformExpressionsUp {
+case UnresolvedAdd(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeAdd(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeAdd(r, l), r.dataType)
+  case (DateType, _) => DateAdd(l, r)
+  case (_, DateType) => DateAdd(r, l)
+  case (_, _) => Add(l, r)
+}
+case UnresolvedSubtract(l, r) => (l.dataType, r.dataType) match {
+  case (TimestampType | DateType | StringType, CalendarIntervalType) =>
+Cast(TimeSub(l, r), l.dataType)
+  case (CalendarIntervalType, TimestampType | DateType | StringType) =>
+Cast(TimeSub(r, l), r.dataType)
+  case (DateType | NullType, DateType) => if 
(conf.usePostgreSQLDialect) {
 
 Review comment:
   Do we need to handle `NullType` here? The `Subtract` should work for null.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date and Timestamp type +/- null should be null as Postgres

2019-11-27 Thread GitBox
cloud-fan commented on a change in pull request #26412: [SPARK-29774][SQL] Date 
and Timestamp type +/- null should be null as Postgres
URL: https://github.com/apache/spark/pull/26412#discussion_r351252061
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
 ##
 @@ -858,14 +858,19 @@ object TypeCoercion {
 b.withNewChildren(Seq(Cast(l, CalendarIntervalType), r))
 
   case Add(l @ DateType(), r @ IntegerType()) => DateAdd(l, r)
+  case Add(l @ DateType(), r @ NullType()) => DateAdd(l, Cast(r, 
IntegerType))
   case Add(l @ IntegerType(), r @ DateType()) => DateAdd(r, l)
+  case Add(l @ NullType(), r @ DateType()) => DateAdd(r, Cast(l, 
IntegerType))
   case Subtract(l @ DateType(), r @ IntegerType()) => DateSub(l, r)
+  case Subtract(l @ DateType(), r @ NullType()) => DateSub(l, Cast(r, 
IntegerType))
   case Subtract(l @ DateType(), r @ DateType()) =>
 if (SQLConf.get.usePostgreSQLDialect) DateDiff(l, r) else 
SubtractDates(l, r)
   case Subtract(l @ TimestampType(), r @ TimestampType()) =>
 SubtractTimestamps(l, r)
   case Subtract(l @ TimestampType(), r @ DateType()) =>
 SubtractTimestamps(l, Cast(r, TimestampType))
+  case Subtract(l @ TimestampType(), r @ NullType()) =>
 
 Review comment:
   how about null - timestamp?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org