sigmod commented on a change in pull request #34053:
URL: https://github.com/apache/spark/pull/34053#discussion_r714443723



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
##########
@@ -2122,6 +2125,68 @@ object RewriteIntersectAll extends Rule[LogicalPlan] {
   }
 }
 
+/**
+ * Replaces logical [[AsOfJoin]] operator using a combination of Join and 
Aggregate operator.
+ *
+ * Input Pseudo-Query:
+ * {{{
+ *    SELECT * FROM left ASOF JOIN right ON (condition, as_of on(left.t, 
right.t), tolerance)
+ * }}}
+ *
+ * Rewritten Query:
+ * {{{
+ *   SELECT left.*, __right__.*
+ *   FROM (
+ *        SELECT
+ *             left.*,
+ *             (
+ *                  SELECT MIN_BY(STRUCT(right.*), left.t - right.t)
+ *                  FROM right
+ *                  WHERE condition AND left.t >= right.t AND right.t >= 
left.t - tolerance
+ *             ) as __right__
+ *        FROM left
+ *        )
+ * }}}
+ */
+object RewriteAsOfJoin extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(
+    _.containsPattern(AS_OF_JOIN), ruleId) {
+    case AsOfJoin(left, right, asOfCondition, condition, orderExpression, 
joinType) =>
+      val conditionWithOuterReference =
+        condition.map(And(_, 
asOfCondition)).getOrElse(asOfCondition).transformUp {
+          case a: AttributeReference if left.outputSet.contains(a) =>
+            OuterReference(a)
+      }
+      val filtered = Filter(conditionWithOuterReference, right)
+
+      val orderExpressionWithOuterReference = orderExpression.transformUp {
+          case a: AttributeReference if left.outputSet.contains(a) =>
+            OuterReference(a)
+        }
+      val rightStruct = CreateStruct(right.output)
+      val nearestRight = MinBy(rightStruct, orderExpressionWithOuterReference)
+        .toAggregateExpression()
+      val aggExpr = Alias(nearestRight, "__nearest_right__")()
+      val aggregate = Aggregate(Seq.empty, Seq(aggExpr), filtered)
+
+      val scalarSubquery = Project(

Review comment:
       Nit: projectWithScalarSubquery ?

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/DataFrameAsOfJoinSuite.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types._
+
+class DataFrameAsOfJoinSuite extends QueryTest
+  with SharedSparkSession
+  with AdaptiveSparkPlanHelper {
+
+  def prepareForAsOfJoin(): (DataFrame, DataFrame) = {
+    val schema1 = StructType(
+      StructField("a", IntegerType, false) ::
+        StructField("b", StringType, false) ::
+        StructField("left_val", StringType, false) :: Nil)
+    val rowSeq1: List[Row] = List(Row(1, "x", "a"), Row(5, "y", "b"), Row(10, 
"z", "c"))
+    val df1 = spark.createDataFrame(rowSeq1.asJava, schema1)
+
+    val schema2 = StructType(
+      StructField("a", IntegerType) ::
+        StructField("b", StringType) ::
+        StructField("right_val", IntegerType) :: Nil)
+    val rowSeq2: List[Row] = List(Row(1, "v", 1), Row(2, "w", 2), Row(3, "x", 
3),
+      Row(6, "y", 6), Row(7, "z", 7))
+    val df2 = spark.createDataFrame(rowSeq2.asJava, schema2)
+
+    (df1, df2)
+  }
+
+  test("as-of join - simple") {
+    val (df1, df2) = prepareForAsOfJoin()
+    checkAnswer(
+      df1.joinAsOf(
+        df2, df1.col("a"), df2.col("a"), usingColumns = Seq.empty,
+        joinType = "left", tolerance = null, allowExactMatches = true, 
direction = "backward"),
+      Seq(
+        Row(1, "x", "a", 1, "v", 1),
+        Row(5, "y", "b", 3, "x", 3),
+        Row(10, "z", "c", 7, "z", 7)
+      )
+    )
+  }
+
+  test("as-of join - usingColumns") {
+    val (df1, df2) = prepareForAsOfJoin()
+    checkAnswer(
+      df1.joinAsOf(df2, df1.col("a"), df2.col("a"), usingColumns = Seq("b"),
+        joinType = "left", tolerance = null, allowExactMatches = true, 
direction = "backward"),
+      Seq(
+        Row(1, "x", "a", null, null, null),
+        Row(5, "y", "b", null, null, null),
+        Row(10, "z", "c", 7, "z", 7)
+      )
+    )
+  }
+
+  test("as-of join - usingColumns, inner") {
+    val (df1, df2) = prepareForAsOfJoin()
+    checkAnswer(
+      df1.joinAsOf(df2, df1.col("a"), df2.col("a"), usingColumns = Seq("b"),
+        joinType = "inner", tolerance = null, allowExactMatches = true, 
direction = "backward"),
+      Seq(
+        Row(10, "z", "c", 7, "z", 7)
+      )
+    )
+  }
+
+  test("as-of join - tolerance = 1") {
+    val (df1, df2) = prepareForAsOfJoin()
+    checkAnswer(
+      df1.joinAsOf(df2, df1.col("a"), df2.col("a"), usingColumns = Seq.empty,
+        joinType = "left", tolerance = lit(1), allowExactMatches = true, 
direction = "backward"),
+      Seq(
+        Row(1, "x", "a", 1, "v", 1),
+        Row(5, "y", "b", null, null, null),
+        Row(10, "z", "c", null, null, null)
+      )
+    )
+  }
+
+  test("as-of join - allowExactMatches = false") {
+    val (df1, df2) = prepareForAsOfJoin()
+    checkAnswer(
+      df1.joinAsOf(df2, df1.col("a"), df2.col("a"), usingColumns = Seq.empty,
+        joinType = "left", tolerance = null, allowExactMatches = false, 
direction = "backward"),
+      Seq(
+        Row(1, "x", "a", null, null, null),

Review comment:
       In the example, non-matches' numeric columns are NaN?




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to