[GitHub] [spark] yaooqinn 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
yaooqinn 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_r354170836
 
 

 ##
 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:
   
https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPDTIMinus.java#L111-121
   ```scala
  // Allowed operations:
   // IntervalYearMonth - IntervalYearMonth = IntervalYearMonth
   // Date - IntervalYearMonth = Date (operands not reversible)
   // Timestamp - IntervalYearMonth = Timestamp (operands not reversible)
   // IntervalDayTime - IntervalDayTime = IntervalDayTime
   // Date - IntervalYearMonth = Timestamp (operands not reversible)
   // Timestamp - IntervalYearMonth = Timestamp (operands not reversible)
   // Timestamp - Timestamp = IntervalDayTime
   // Date - Date = IntervalDayTime
   // Timestamp - Date = IntervalDayTime (operands reversible)
   // Date - Int = Date
   ```
   Hive's behavior is more convictive, we can check this 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] yaooqinn 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
yaooqinn 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_r354164472
 
 

 ##
 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:
   yea, but follows the behavior before 
https://github.com/apache/spark/pull/26412/files/c84d46ea6d384dcb1f442ca54abad48e59c92bb3#diff-383a8cdd0a9c58cae68e0a79295520a3L846


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] yaooqinn 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
yaooqinn 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_r353588331
 
 

 ##
 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:
   done


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] yaooqinn 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
yaooqinn 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_r353287869
 
 

 ##
 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:
   One way is to prohibit implicit cast for DateAdd and DateSub, which make 
them more conformable with pgsql and hive. the other way is to take care of the 
types in our resolving process, but this will cause date add/sub functions 
different from `+/-`.
   
   I took a quick look at Hive's Jira for these functions, they are from MySQL, 
the MySQL original ones only take interval as the second arguments


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] yaooqinn 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
yaooqinn 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_r353273100
 
 

 ##
 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:
   No, our date_add/date_sub is werid too, Hive and Postgres only support type 
<=(int4)


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] yaooqinn 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
yaooqinn 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_r353222500
 
 

 ##
 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:
   no, I'll remove this cast, it's legacy


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] yaooqinn 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
yaooqinn 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_r353221948
 
 

 ##
 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:
   yea, I will change this back, I'am suffering tests fail with 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] yaooqinn 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
yaooqinn 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_r353055059
 
 

 ##
 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:
   Yes, type coercion will take over after we resolve 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] yaooqinn 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
yaooqinn 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_r353053467
 
 

 ##
 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:
   Yes, type coercion will take over after we resolve 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] yaooqinn 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
yaooqinn 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_r353050305
 
 

 ##
 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:
   Er...  here we have no problem, just reply to this comment
   >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] yaooqinn 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
yaooqinn 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_r353032872
 
 

 ##
 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:
   neither for pg,
   ```
   postgres=# select cast(1 as interval);
   ERROR:  cannot cast type integer to interval
   LINE 1: select cast(1 as interval);
  ^
   postgres=# select cast(1 as date);
   ERROR:  cannot cast type integer to date
   LINE 1: select cast(1 as date);
  ^
   postgres=# select interval '1' day  + 1;
   ERROR:  operator does not exist: interval + integer
   LINE 1: select interval '1' day  + 1;
^
   ```


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] yaooqinn 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
yaooqinn 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_r353032252
 
 

 ##
 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:
   we do not support cast int to interval, so interval + int is not supported.


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] yaooqinn 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
yaooqinn 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_r353007005
 
 

 ##
 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:
   ```
   org.apache.spark.sql.AnalysisException
   +cannot resolve '(INTERVAL '1 days' + 1)' due to data type mismatch: 
differing types in '(INTERVAL '1 days' + 1)'
   ```


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] yaooqinn 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
yaooqinn 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_r353006365
 
 

 ##
 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:
   they all go to `ADD` then fail with type checking


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] yaooqinn 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
yaooqinn 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_r353005158
 
 

 ##
 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:
   you missed `interval + 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] yaooqinn 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
yaooqinn 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_r352963519
 
 

 ##
 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:
   From hive
   ```
   DATE_ADD() only takes TINYINT/SMALLINT/INT types as second argument, got 
DOUBLE
   ```


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] yaooqinn 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
yaooqinn 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_r352604626
 
 

 ##
 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 u @ UnresolvedAdd(l, r) if u.childrenResolved => (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 u @ UnresolvedSubtract(l, r) if u.childrenResolved => 
(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 u @ UnresolvedMultiply(l, r) if u.childrenResolved => 
(l.dataType, r.dataType) match {
+  case (CalendarIntervalType, _: NumericType | NullType) => 
MultiplyInterval(l, r)
+  case (_: NumericType | NullType, CalendarIntervalType) => 
MultiplyInterval(r, l)
 
 Review comment:
   `case (_, CalendarIntervalType) => MultiplyInterval(r, l)` might be enough 
for this case.


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] yaooqinn 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
yaooqinn 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_r352602498
 
 

 ##
 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 u @ UnresolvedAdd(l, r) if u.childrenResolved => (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 u @ UnresolvedSubtract(l, r) if u.childrenResolved => 
(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 u @ UnresolvedMultiply(l, r) if u.childrenResolved => 
(l.dataType, r.dataType) match {
+  case (CalendarIntervalType, _: NumericType | NullType) => 
MultiplyInterval(l, r)
+  case (_: NumericType | NullType, CalendarIntervalType) => 
MultiplyInterval(r, l)
 
 Review comment:
   Here is another thing to discuss, do we have to only support `interval * 
numeric` and`interval / numeric`, this is not same with other type coercion 
rules, e.g. `1 / '2'` is valid, so `interval 1 day / '2'` should be valid too.
   
   In PostgreSQL, also valid
   ```
   postgres=# select interval '1' day * 2;
?column?
   --
2 days
   (1 row)
   
   postgres=# select interval '1' day * '2';
?column?
   --
2 days
   (1 row)
   
   postgres=# select interval '1' day / '2';
?column?
   --
12:00:00
   (1 row)
   ```


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] yaooqinn 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
yaooqinn 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_r352598951
 
 

 ##
 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:
   Update a bit
   ```scala
 /**
  * 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]].
  */
   ```


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] yaooqinn 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
yaooqinn 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_r352580043
 
 

 ##
 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:
   Em.. I guess that you are pinging @maropu?


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] yaooqinn 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
yaooqinn 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_r352485983
 
 

 ##
 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)
 
 Review comment:
   I notice that `TimeSub` is replaceable by `TimeAdd(l, UnaryMinus(r))`, which 
make it  useless


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] yaooqinn 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
yaooqinn 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_r352484296
 
 

 ##
 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:
   Actually not,  subtract(null, date) will not pass type checking


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] yaooqinn 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
yaooqinn 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_r352482302
 
 

 ##
 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:
   yea, the result same but do not semantic equal, is that OK? 


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] yaooqinn 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
yaooqinn 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_r352479715
 
 

 ##
 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:
   Hi @cloud-fan, when I was implementing this, I got a bit confused.
   Our date_sub/date_add actually support implicit type cast, that is
   ```
   select date_add(date'1900-01-01', 1);
   select date_add(date'1900-01-01', 1.2);
   select date_add(date'1900-01-01', "1.2");
   ```
   are supported, and result the same.
   
   but while using `date'1900-01-01' + 1`, we only support int. Is this 
nonconforming?


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] yaooqinn 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
yaooqinn 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_r351269771
 
 

 ##
 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:
   yes, we need them too. checked with pg


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] yaooqinn 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
yaooqinn 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_r351248468
 
 

 ##
 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()) =>
 
 Review comment:
   FYI, `add` with numeric type and null type is also handled in `TypeCoercion` 
too


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] yaooqinn 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
yaooqinn 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_r351240739
 
 

 ##
 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()) =>
 
 Review comment:
   If so, we might leave a trivial bug here if we set 
`spark.sql.optimizer.maxIterations=1`, it will not be transformed to `DateAdd `


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