This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 674fd7a6f4d7 [SPARK-51565][SQL] Support SQL parameters in window frame 
clause
674fd7a6f4d7 is described below

commit 674fd7a6f4d70ff1c7cf9c47519f869bfbb94811
Author: Dušan Tišma <dusan.ti...@databricks.com>
AuthorDate: Thu Mar 20 10:20:13 2025 +0800

    [SPARK-51565][SQL] Support SQL parameters in window frame clause
    
    ### What changes were proposed in this pull request?
    This PR allows for SQL named and positional parameters to be used within 
the window frame clause of window expressions.
    
    ### Why are the changes needed?
    SQL parameters should be supported for this use case.
    
    ### Does this PR introduce _any_ user-facing change?
    Yes, users can now use sql parameters in window frame clauses.
    
    ### How was this patch tested?
    Unit tests were added testing this case.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #50328 from dusantism-db/window-function-params.
    
    Authored-by: Dušan Tišma <dusan.ti...@databricks.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../spark/sql/catalyst/parser/AstBuilder.scala     |  4 +-
 .../sql/execution/SQLWindowFunctionSuite.scala     | 45 ++++++++++++++++++++++
 2 files changed, 48 insertions(+), 1 deletion(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 4dafc83b9923..1b268be361a3 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -3153,7 +3153,9 @@ class AstBuilder extends DataTypeAstBuilder
   override def visitFrameBound(ctx: FrameBoundContext): Expression = 
withOrigin(ctx) {
     def value: Expression = {
       val e = expression(ctx.expression)
-      validate(e.resolved && e.foldable, "Frame bound value must be a 
literal.", ctx)
+      validate(
+        e.resolved && e.foldable || e.isInstanceOf[Parameter],
+        "Frame bound value must be a literal.", ctx)
       e
     }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
index eec396b2e399..aa619c5cde87 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala
@@ -502,4 +502,49 @@ class SQLWindowFunctionSuite extends QueryTest with 
SharedSparkSession {
 
     spark.catalog.dropTempView("nums")
   }
+
+  test("sql parameters in window frame clause") {
+    val data = Seq(
+      WindowData(1, "d", 10),
+      WindowData(2, "a", 6),
+      WindowData(3, "b", 7),
+      WindowData(4, "b", 8),
+      WindowData(5, "c", 9),
+      WindowData(6, "c", 11)
+    )
+    val expected = Seq(
+      Row(11),
+      Row(12),
+      Row(15),
+      Row(6),
+      Row(6),
+      Row(9)
+    )
+
+    withTempView("windowData") {
+      
sparkContext.parallelize(data).toDF().createOrReplaceTempView("windowData")
+
+      // Named parameters.
+      val namedParamSql = """
+        |SELECT
+        |  SUM(month) OVER (ORDER BY month ROWS BETWEEN CURRENT ROW AND 
:param1 FOLLOWING)
+        |FROM windowData
+      """.stripMargin
+      checkAnswer(spark.sql(namedParamSql, Map("param1" -> 2)), expected)
+
+      // Positional parameters.
+      val postParamSql = """
+        |SELECT
+        |  SUM(month) OVER (ORDER BY month ROWS BETWEEN CURRENT ROW AND ? 
FOLLOWING)
+        |FROM windowData
+      """.stripMargin
+      checkAnswer(spark.sql(postParamSql, Array(2)), expected)
+
+      // Wrong type of parameter.
+      val e = intercept[AnalysisException] {
+        spark.sql(namedParamSql, Map("param1" -> "abc")).collect()
+      }
+      
assert(e.errorClass.contains("DATATYPE_MISMATCH.SPECIFIED_WINDOW_FRAME_UNACCEPTED_TYPE"))
+    }
+  }
 }


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

Reply via email to