You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/22 13:39:59 UTC

[GitHub] [spark] MaxGekk opened a new pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

MaxGekk opened a new pull request #31929:
URL: https://github.com/apache/spark/pull/31929


   ### What changes were proposed in this pull request?
   1. Add new expression `MultiplyYMInterval` which multiplies a `YearMonthIntervalType` expression by a `NumericType` expression including ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.
   2. Extend binary arithmetic rules to support `numeric * year-month interval` and `year-month interval * numeric`.
   
   ### Why are the changes needed?
   To conform the ANSI SQL standard which requires such operation over year-month intervals:
   <img width="667" alt="Screenshot 2021-03-22 at 16 33 16" src="https://user-images.githubusercontent.com/1580697/111997810-77d1eb80-8b2c-11eb-951d-e43911d9c5db.png">
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   By running new tests:
   ```
   $ build/sbt "test:testOnly *IntervalExpressionsSuite"
   $ build/sbt "test:testOnly *ColumnExpressionSuite"
   ```


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599392263



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {

Review comment:
       Strings are also accepted then?




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] tanelk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
tanelk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r598789413



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
##########
@@ -274,4 +276,26 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       checkException(weeks = Int.MaxValue)
     }
   }
+
+  test("SPARK-34824: multiply year-month interval by numeric") {

Review comment:
       Should we also add `checkConsistencyBetweenInterpretedAndCodegen`




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804323683


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136346/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805053770


   GA passed. Merging to master.
   Thank you @yaooqinn @cloud-fan for your review.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804483533


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136358/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599348187



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       SGTM




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804990380


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136399/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804344747


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804483533


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136358/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r598880189



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
##########
@@ -274,4 +276,26 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       checkException(weeks = Int.MaxValue)
     }
   }
+
+  test("SPARK-34824: multiply year-month interval by numeric") {

Review comment:
       @tanelk Thank you for review. I will add the check.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804780605


   **[Test build #136399 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136399/testReport)** for PR 31929 at commit [`6acff97`](https://github.com/apache/spark/commit/6acff97182d6d890dd0148383656ae22684caa58).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804780605


   **[Test build #136399 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136399/testReport)** for PR 31929 at commit [`6acff97`](https://github.com/apache/spark/commit/6acff97182d6d890dd0148383656ae22684caa58).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804323683


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136346/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804148823


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40930/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804427666


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40945/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804882120


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804954245


   **[Test build #136394 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136394/testReport)** for PR 31929 at commit [`2d21d57`](https://github.com/apache/spark/commit/2d21d5796bd7cc1472820711b73c20a76bff2451).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804290773


   **[Test build #136358 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136358/testReport)** for PR 31929 at commit [`1087254`](https://github.com/apache/spark/commit/1087254e5dd9ce3cd9cefe1a05c222db5c156242).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804094196


   **[Test build #136346 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136346/testReport)** for PR 31929 at commit [`1ec715f`](https://github.com/apache/spark/commit/1ec715f3f9cb17f1f9dc8185ef9ae7513e35b9e1).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804816414


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40977/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804207281


   @cloud-fan @yaooqinn Could you review this PR, please.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804375387


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804465984


   **[Test build #136358 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136358/testReport)** for PR 31929 at commit [`1087254`](https://github.com/apache/spark/commit/1087254e5dd9ce3cd9cefe1a05c222db5c156242).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804331394


   **[Test build #136361 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136361/testReport)** for PR 31929 at commit [`47dc12d`](https://github.com/apache/spark/commit/47dc12d3e5666f0ad159aff96f27dc077728c660).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599367187



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       I had to override `toString()` because `symbol()` is defined in `BinaryOperator` which I cannot use here as it requires both `left` and `right` must have the same type.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804983356


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136394/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804508165


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136361/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599346653



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       For example, the `Multiply` expression does not override `prettyName()`. It only overrides `symbol`: https://github.com/apache/spark/blob/143303147b64c0ff7f57c217f9d4b9de2cbff649/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala#L375




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804816414


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40977/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804742634


   **[Test build #136394 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136394/testReport)** for PR 31929 at commit [`2d21d57`](https://github.com/apache/spark/commit/2d21d5796bd7cc1472820711b73c20a76bff2451).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599393027



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
##########
@@ -274,4 +277,32 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       checkException(weeks = Int.MaxValue)
     }
   }
+
+  test("SPARK-34824: multiply year-month interval by numeric") {
+    Seq(
+      (Period.ofMonths(0), 10) -> Period.ofMonths(0),
+      (Period.ofMonths(10), 0L) -> Period.ofMonths(0),
+      (Period.ofYears(100), -1.toByte) -> Period.ofYears(-100),
+      (Period.ofMonths(12), 0.3f) -> Period.ofMonths(4),
+      (Period.ofYears(-1000), 0.3d) -> Period.ofYears(-300),
+      (Period.ofYears(9999), 0.0001d) -> Period.ofYears(1),
+      (Period.ofYears(9999), BigDecimal(0.0001)) -> Period.ofYears(1)
+    ).foreach { case ((period, num), expected) =>

Review comment:
       add cases for nulls and NaNs?




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk closed pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk closed pull request #31929:
URL: https://github.com/apache/spark/pull/31929


   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] yaooqinn commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599405553



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {

Review comment:
       ah sorry, I missed that 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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804796499


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804351324


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804890597


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40983/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804094196


   **[Test build #136346 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136346/testReport)** for PR 31929 at commit [`1ec715f`](https://github.com/apache/spark/commit/1ec715f3f9cb17f1f9dc8185ef9ae7513e35b9e1).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804314916


   **[Test build #136346 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136346/testReport)** for PR 31929 at commit [`1ec715f`](https://github.com/apache/spark/commit/1ec715f3f9cb17f1f9dc8185ef9ae7513e35b9e1).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599344204



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       I think of `override def symbol: String = "*"` since it is used in `toString`:
   ```scala
     def symbol: String
   
     def sqlOperator: String = symbol
   
     override def toString: String = s"($left $sqlOperator $right)"
   ```
   and leave `prettyName` as IS (not override it in the new expression)




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804963804


   jenkins, retest this, please


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804990380


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136399/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805122210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40998/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599322992



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       it's not a function but operator, I think we should override `toString` instead: `override def toString: String = s"($left * $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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599398304



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {

Review comment:
       yes, I checked that in a test.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804986788


   **[Test build #136414 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136414/testReport)** for PR 31929 at commit [`6acff97`](https://github.com/apache/spark/commit/6acff97182d6d890dd0148383656ae22684caa58).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804331394


   **[Test build #136361 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136361/testReport)** for PR 31929 at commit [`47dc12d`](https://github.com/apache/spark/commit/47dc12d3e5666f0ad159aff96f27dc077728c660).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599426236



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {

Review comment:
       Sure, I am adding now.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599326963



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       yep. You are right. BTW, `s"($left * $right)"` is not completely correct because the expression can be used in `num * year-month interval` and in `year-month interval * num`. But here we lost the info about the order. 




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805002670


   jenkins, retest this, please


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804987738


   **[Test build #136399 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136399/testReport)** for PR 31929 at commit [`6acff97`](https://github.com/apache/spark/commit/6acff97182d6d890dd0148383656ae22684caa58).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804141014


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804742634


   **[Test build #136394 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136394/testReport)** for PR 31929 at commit [`2d21d57`](https://github.com/apache/spark/commit/2d21d5796bd7cc1472820711b73c20a76bff2451).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804148823


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40930/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599344204



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       I think of `override def symbol: String = "*"` since it is used in `toString`:
   ```scala
     def symbol: String
   
     def sqlOperator: String = symbol
   
     override def toString: String = s"($left $sqlOperator $right)"
   ```
   and leave `prettyName` as IS.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599343560



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/intervalExpressions.scala
##########
@@ -249,3 +249,49 @@ case class MakeInterval(
 
   override def prettyName: String = "make_interval"
 }
+
+// Multiply an year-month interval by a numeric
+case class MultiplyYMInterval(
+    interval: Expression,
+    num: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes with NullIntolerant with Serializable {
+  override def left: Expression = interval
+  override def right: Expression = num
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(YearMonthIntervalType, NumericType)
+  override def dataType: DataType = YearMonthIntervalType
+
+  @transient
+  private lazy val evalFunc: (Int, Any) => Any = right.dataType match {
+    case ByteType | ShortType | IntegerType => (months: Int, num) =>
+      Math.multiplyExact(months, num.asInstanceOf[Number].intValue())
+    case LongType => (months: Int, num) =>
+      Math.toIntExact(Math.multiplyExact(months, num.asInstanceOf[Long]))
+    case FloatType | DoubleType => (months: Int, num) =>
+      Math.toIntExact(Math.round(months * num.asInstanceOf[Number].doubleValue()))
+    case _: DecimalType => (months: Int, num) =>
+      val decimalRes = ((new Decimal).set(months) * num.asInstanceOf[Decimal]).toJavaBigDecimal
+      decimalRes.setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()
+  }
+
+  override def nullSafeEval(interval: Any, num: Any): Any = {
+    evalFunc(interval.asInstanceOf[Int], num)
+  }
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = right.dataType match {
+    case ByteType | ShortType | IntegerType =>
+      defineCodeGen(ctx, ev, (m, n) => s"java.lang.Math.multiplyExact($m, $n)")
+    case LongType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.multiplyExact($m, $n))")
+    case FloatType | DoubleType =>
+      val jlm = classOf[Math].getName
+      defineCodeGen(ctx, ev, (m, n) => s"$jlm.toIntExact($jlm.round($m * (double)$n))")
+    case _: DecimalType =>
+      defineCodeGen(ctx, ev, (m, n) =>
+        s"((new Decimal()).set($m).$$times($n)).toJavaBigDecimal()" +
+        ".setScale(0, java.math.RoundingMode.HALF_UP).intValueExact()")
+  }
+
+  override def prettyName: String = "multiply_ym_interval"

Review comment:
       I think it's not a big deal and we don't have to retain the order.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804890597


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40983/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805033652


   **[Test build #136416 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136416/testReport)** for PR 31929 at commit [`6acff97`](https://github.com/apache/spark/commit/6acff97182d6d890dd0148383656ae22684caa58).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804983356


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136394/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805122210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40998/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804494707


   **[Test build #136361 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136361/testReport)** for PR 31929 at commit [`47dc12d`](https://github.com/apache/spark/commit/47dc12d3e5666f0ad159aff96f27dc077728c660).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804351478


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40942/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804404193


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804859195


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804351478


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40942/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805161975


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41000/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804148770


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804427666


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/40945/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804803105


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


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] MaxGekk commented on a change in pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on a change in pull request #31929:
URL: https://github.com/apache/spark/pull/31929#discussion_r599426474



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/IntervalExpressionsSuite.scala
##########
@@ -274,4 +277,32 @@ class IntervalExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
       checkException(weeks = Int.MaxValue)
     }
   }
+
+  test("SPARK-34824: multiply year-month interval by numeric") {
+    Seq(
+      (Period.ofMonths(0), 10) -> Period.ofMonths(0),
+      (Period.ofMonths(10), 0L) -> Period.ofMonths(0),
+      (Period.ofYears(100), -1.toByte) -> Period.ofYears(-100),
+      (Period.ofMonths(12), 0.3f) -> Period.ofMonths(4),
+      (Period.ofYears(-1000), 0.3d) -> Period.ofYears(-300),
+      (Period.ofYears(9999), 0.0001d) -> Period.ofYears(1),
+      (Period.ofYears(9999), BigDecimal(0.0001)) -> Period.ofYears(1)
+    ).foreach { case ((period, num), expected) =>

Review comment:
       Sure, I am adding the checks now.




-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-805161975


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/41000/
   


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] SparkQA commented on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804290773


   **[Test build #136358 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/136358/testReport)** for PR 31929 at commit [`1087254`](https://github.com/apache/spark/commit/1087254e5dd9ce3cd9cefe1a05c222db5c156242).


-- 
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:
users@infra.apache.org



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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31929: [SPARK-34824][SQL] Support multiply an year-month interval by a numeric

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31929:
URL: https://github.com/apache/spark/pull/31929#issuecomment-804508165


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/136361/
   


-- 
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:
users@infra.apache.org



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