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 2022/05/27 13:41:21 UTC

[GitHub] [spark] MaxGekk commented on a diff in pull request #36654: [SPARK-39259] Evaluate timestamps consistently in subqueries

MaxGekk commented on code in PR #36654:
URL: https://github.com/apache/spark/pull/36654#discussion_r883621685


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala:
##########
@@ -72,30 +73,34 @@ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] {
  * Computes the current date and time to make sure we return the same result in a single query.
  */
 object ComputeCurrentTime extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan): LogicalPlan = {
-    val currentDates = mutable.Map.empty[String, Literal]
-    val timeExpr = CurrentTimestamp()
-    val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
-    val currentTime = Literal.create(timestamp, timeExpr.dataType)
+  def apply(plan: LogicalPlan): LogicalPlan = applyWithTimestamp(plan, Instant.now())
+
+  /** Required to build custom rules for commands that do not keep sub-plans as children in Delta */
+  def applyWithTimestamp(plan: LogicalPlan, instant: Instant): LogicalPlan = {
+    val currentTimestamp = instantToMicros(instant)

Review Comment:
   Maybe, just invoke DateTimeUtils.currentTimestamp()



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala:
##########
@@ -72,30 +73,34 @@ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] {
  * Computes the current date and time to make sure we return the same result in a single query.
  */
 object ComputeCurrentTime extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan): LogicalPlan = {
-    val currentDates = mutable.Map.empty[String, Literal]
-    val timeExpr = CurrentTimestamp()
-    val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
-    val currentTime = Literal.create(timestamp, timeExpr.dataType)
+  def apply(plan: LogicalPlan): LogicalPlan = applyWithTimestamp(plan, Instant.now())
+
+  /** Required to build custom rules for commands that do not keep sub-plans as children in Delta */

Review Comment:
   In Delta? Do you refer to open source Delta or Databricks Delta. In any case, we shouldn't do that, I think.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala:
##########
@@ -72,30 +73,34 @@ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] {
  * Computes the current date and time to make sure we return the same result in a single query.
  */
 object ComputeCurrentTime extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan): LogicalPlan = {
-    val currentDates = mutable.Map.empty[String, Literal]
-    val timeExpr = CurrentTimestamp()
-    val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
-    val currentTime = Literal.create(timestamp, timeExpr.dataType)
+  def apply(plan: LogicalPlan): LogicalPlan = applyWithTimestamp(plan, Instant.now())

Review Comment:
   Could you inline `applyWithTimestamp`, please, since it is used only once here.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala:
##########
@@ -72,30 +73,34 @@ object RewriteNonCorrelatedExists extends Rule[LogicalPlan] {
  * Computes the current date and time to make sure we return the same result in a single query.
  */
 object ComputeCurrentTime extends Rule[LogicalPlan] {
-  def apply(plan: LogicalPlan): LogicalPlan = {
-    val currentDates = mutable.Map.empty[String, Literal]
-    val timeExpr = CurrentTimestamp()
-    val timestamp = timeExpr.eval(EmptyRow).asInstanceOf[Long]
-    val currentTime = Literal.create(timestamp, timeExpr.dataType)
+  def apply(plan: LogicalPlan): LogicalPlan = applyWithTimestamp(plan, Instant.now())
+
+  /** Required to build custom rules for commands that do not keep sub-plans as children in Delta */
+  def applyWithTimestamp(plan: LogicalPlan, instant: Instant): LogicalPlan = {
+    val currentTimestamp = instantToMicros(instant)
+    val currentTime = Literal.create(currentTimestamp, TimestampType)
     val timezone = Literal.create(conf.sessionLocalTimeZone, StringType)
-    val localTimestamps = mutable.Map.empty[String, Literal]
 
-    plan.transformAllExpressionsWithPruning(_.containsPattern(CURRENT_LIKE)) {
-      case currentDate @ CurrentDate(Some(timeZoneId)) =>
-        currentDates.getOrElseUpdate(timeZoneId, {
-          Literal.create(currentDate.eval().asInstanceOf[Int], DateType)
-        })
-      case CurrentTimestamp() | Now() => currentTime
-      case CurrentTimeZone() => timezone
-      case localTimestamp @ LocalTimestamp(Some(timeZoneId)) =>
-        localTimestamps.getOrElseUpdate(timeZoneId, {
-          Literal.create(localTimestamp.eval().asInstanceOf[Long], TimestampNTZType)
-        })
+    def transformCondition(treePatternbits: TreePatternBits): Boolean = {
+      treePatternbits.containsPattern(CURRENT_LIKE)
+    }
+
+    plan.transformDownWithSubqueries(transformCondition) {
+      case subQuery =>
+        subQuery.transformAllExpressionsWithPruning(transformCondition) {
+          case cd: CurrentDate =>
+            Literal.create(
+              localDateToDays(instant.atZone(cd.zoneId).toLocalDate).asInstanceOf[Int], DateType)

Review Comment:
   Can't you re-use DateTimeUtils.microsToDays() ?



-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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