You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "milastdbx (via GitHub)" <gi...@apache.org> on 2023/11/30 16:34:13 UTC

[PR] EXECUTE IMMEDIATE SQL support [spark]

milastdbx opened a new pull request, #44093:
URL: https://github.com/apache/spark/pull/44093

   
   ### What changes were proposed in this pull request?
   
   Introducing new EXECUTE IMMEDIATE syntax to support parameterized queries from within SQL.
   
   This API executes query passed as string with arguments.
   
   Other DBs that support this:
    - [Oracle](https://docs.oracle.com/cd/B13789_01/appdev.101/b10807/13_elems017.htm)
    - [Snowflake](https://docs.snowflake.com/en/sql-reference/sql/execute-immediate)
    - [PgSql](https://www.postgresql.org/docs/current/ecpg-sql-execute-immediate.html#:~:text=Description,statement%2C%20without%20retrieving%20result%20rows.)
   
   ### Why are the changes needed?
   Often time queries are constructed as a result of other queries. We need a way to dynamically construct a query and execute it without leaving SQL API.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, it exposes new syntax
   
   
   ### How was this patch tested?
   Golden files
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No
   


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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1422234166


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2261,6 +2273,12 @@
     },
     "sqlState" : "42000"
   },
+  "INVALID_STATEMENT_FOR_EXECUTE_INTO" : {
+    "message" : [
+      "The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement isn't: <sqlString>."

Review Comment:
   Given that this is error message, isn't Failed redundant ?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427904038


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -558,6 +560,73 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
   }
 
   /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {
+    // Because of how parsing rules are written, we know that either
+    // queryParam or targetVariable is non null - hence use Either to represent this.
+    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(string(visitStringLit(sl))))

Review Comment:
   Shall we advance the check for `queryString`?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]])

Review Comment:
   +1



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -58,7 +59,8 @@ import org.apache.spark.util.random.RandomSampler
  * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
  * TableIdentifier.
  */
-class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
+class AstBuilder
+    extends DataTypeAstBuilder with SQLConfHelper with Logging {

Review Comment:
   +1



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1428663512


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -558,6 +560,73 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
   }
 
   /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {
+    // Because of how parsing rules are written, we know that either
+    // queryParam or targetVariable is non null - hence use Either to represent this.
+    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(string(visitStringLit(sl))))

Review Comment:
   The code parse the query string within `SubstituteExecuteImmediate` now.
   It seems we can parse it at `AstBuilder` so as avoid analysis stage.
   



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427843998


##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+  "sqlState" : "07501",
+  "messageParameters" : {
+    "sqlString" : "INSERT INTO x VALUES (?)"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNRESOLVED_VARIABLE",
+  "sqlState" : "42883",
+  "messageParameters" : {
+    "searchPath" : "`SYSTEM`.`SESSION`",
+    "variableName" : "`id`"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where ? = id and :first = name' USING 1, 'name2' as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+  "sqlState" : "42613"
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELCT F(
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'''",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+a'
+
+EXECUTE IMMEDIATE b
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'a'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'na'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'('",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO a, b USING 10
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'name1'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 81,

Review Comment:
   similar things seems to be happening to SET.
   Ill lookup the fix, but does not seem trivial as this happens at later stages
   
   <img width="1728" alt="image" src="https://github.com/apache/spark/assets/150366084/49abb4f2-25ba-406e-b2bd-f2f716e0edd6">
   



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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1431067318


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw QueryCompilationErrors.unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType)
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw QueryCompilationErrors.invalidQueryMixedQueryParameters()
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])

Review Comment:
   Seems don't need to iterate over `expressions` twice. Just add a default case to `collect` and throw the exception there.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw QueryCompilationErrors.unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType)
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw QueryCompilationErrors.invalidQueryMixedQueryParameters()
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(aliases))
+          }
+        }
+
+        if (targetVariables.nonEmpty) {
+          SetVariable(targetVariables, queryPlan)
+        } else { queryPlan }
+    }
+
+  private def parseStatement(
+      queryString: String,
+      targetVariables: Seq[Expression]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    val plan = if (targetVariables.nonEmpty) {
+      try {
+        catalogManager.v1SessionCatalog.parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+          // Plan was successfully parsed, but query wasn't - throw.
+          throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString)
+      }
+    } else {
+      catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+    }
+
+    // do not allow nested execute immediate
+    if (plan.containsPattern(EXECUTE_IMMEDIATE)) {
+      throw QueryCompilationErrors.nestedExecuteImmediate(queryString)
+    }
+
+    plan
+  }
+
+  private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = {
+    lookupVariable(nameParts) match {
+      case Some(variable) => variable.copy(canFold = false)
+      case _ =>
+        throw QueryCompilationErrors
+          .unresolvedVariableError(nameParts, Seq("SYSTEM", "SESSION"), expr.origin)

Review Comment:
   Could you use `CatalogManager.SYSTEM_CATALOG_NAME` and `CatalogManager.SESSION_NAMESPACE`



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


Re: [PR] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1411948049


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2261,6 +2267,12 @@
     },
     "sqlState" : "42000"
   },
+  "INVALID_STATEMENT_FOR_EXECUTE_INTO" : {
+    "message" : [
+      "The INTO clause of EXECUTE IMMEDIATE is only valid for queries. The statement starting with: <sqlString> is a(n) <stmtType>."

Review Comment:
   I think it doesn't matter what's type of the invalid query, just say:
   ```
   ...is only valid for queries, but the given SQL statement isn't: <sqlStmt>.
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2338,6 +2350,12 @@
     ],
     "sqlState" : "42000"
   },
+  "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : {
+    "message" : [
+      "Variable passed query must be of type STRING"

Review Comment:
   minor but types must be quoted:
   ```suggestion
         "Variable type must be a \"STRING\" but got <varType>."
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala:
##########
@@ -113,7 +113,8 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase {
   private def checkArgs(args: Iterable[(String, Expression)]): Unit = {
     def isNotAllowed(expr: Expression): Boolean = expr.exists {
       case _: Literal | _: CreateArray | _: CreateNamedStruct |
-        _: CreateMap | _: MapFromArrays |  _: MapFromEntries => false
+          _: CreateMap | _: MapFromArrays |  _: MapFromEntries | _: VariableReference => false

Review Comment:
   Please, revert indentation back.



##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4:
##########
@@ -217,6 +217,7 @@ HOURS: 'HOURS';
 IDENTIFIER_KW: 'IDENTIFIER';
 IF: 'IF';
 IGNORE: 'IGNORE';
+IMMEDIATE: 'IMMEDIATE';

Review Comment:
   Is it reserved keyword or not. You should mention this in a doc.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   Is it used multiple times? If not, please, inline it.



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


Re: [PR] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1412764471


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = query match {
+        case Left(v) => v
+        case Right(u) =>
+          var varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.eval(null).isInstanceOf[UTF8String]) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map.empty)
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+
+      val plan = CatalystSqlParser.parsePlan(queryString);
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.map(_.collect { case n: PosParameter => n }).flatten
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.map(_.collect{ case n: NamedParameter => n }).flatten
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_PARAMETRIZED_QUERY",
+          messageParameters = Map.empty)
+      } else {
+        if (posNodes.nonEmpty) {
+          // Add aggregation or a project.
+          PosParameterizedQuery(
+            plan,
+            expressions)
+        }
+        else {
+          val namedExpressions = expressions.collect {
+            case (e: NamedExpression) => e
+          }
+
+          NameParameterizedQuery(
+            plan,
+            namedExpressions.map(_.name),
+            namedExpressions)
+        }
+      }
+
+      targetVariablesOpt.map (
+        expressions => {
+          queryPlan.find {
+            c => c.containsPattern(COMMAND)
+          }.map { c =>
+            throw new AnalysisException(
+              errorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+              messageParameters = Map(
+                "sqlString" -> queryString,
+                // TODO: change command with proper statement type
+                "stmtType" -> "Command"
+              ))
+          }

Review Comment:
   Isn't the same as:
   ```suggestion
             if (queryPlan.exists(_.containsPattern(COMMAND))) {
               throw new AnalysisException(
                 errorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO",
                 messageParameters = Map(
                   "sqlString" -> queryString,
                   // TODO: change command with proper statement type
                   "stmtType" -> "Command"))
             }
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = query match {
+        case Left(v) => v
+        case Right(u) =>
+          var varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.eval(null).isInstanceOf[UTF8String]) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map.empty)
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString

Review Comment:
   It has been already evaluated above, can you reuse the result, please.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = query match {
+        case Left(v) => v
+        case Right(u) =>
+          var varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.eval(null).isInstanceOf[UTF8String]) {

Review Comment:
   Do you really have to eval? If you need the type, you can just call `dataType`



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = query match {
+        case Left(v) => v
+        case Right(u) =>
+          var varReference = lookupVariable(u.nameParts) match {

Review Comment:
   ```suggestion
             val varReference = lookupVariable(u.nameParts) match {
   ```



##########
docs/sql-ref-syntax-aux-exec-imm.md:
##########
@@ -0,0 +1,86 @@
+---
+layout: global
+title: EXECUTE IMMEDIATE
+displayTitle: EXECUTE IMMEDIATE
+license: |
+  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.
+---
+
+### Description
+
+Executes a sql statement provided as a `STRING`, optionally passing `arg_exprN` to parameter markers and assigning the results to `var_nameN`.
+
+### Syntax
+
+```sql
+EXECUTE IMMEDIATE sql_string
+        [ INTO (var_name [, …] ) ]
+        [ USING ( arg_expr [ AS ] [alias] [, …] ) ]
+        ```

Review Comment:
   fix indentation



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = query match {
+        case Left(v) => v
+        case Right(u) =>
+          var varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.eval(null).isInstanceOf[UTF8String]) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map.empty)
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+
+      val plan = CatalystSqlParser.parsePlan(queryString);
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.map(_.collect { case n: PosParameter => n }).flatten
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.map(_.collect{ case n: NamedParameter => n }).flatten
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_PARAMETRIZED_QUERY",
+          messageParameters = Map.empty)
+      } else {
+        if (posNodes.nonEmpty) {
+          // Add aggregation or a project.
+          PosParameterizedQuery(
+            plan,
+            expressions)
+        }
+        else {

Review Comment:
   ```suggestion
           } else {
   ```



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1414266503


##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4:
##########
@@ -217,6 +217,7 @@ HOURS: 'HOURS';
 IDENTIFIER_KW: 'IDENTIFIER';
 IF: 'IF';
 IGNORE: 'IGNORE';
+IMMEDIATE: 'IMMEDIATE';

Review Comment:
   It should be reserved keyword, right ?
   I should mention that in `sql-ref-syntax-aux-exec-imm.md`?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419130125


##########
docs/sql-ref-ansi-compliance.md:
##########
@@ -9,9 +9,9 @@ license: |
   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
- 

Review Comment:
   accidentaly pushed this, my bad, will revert



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419295232


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {

Review Comment:
   Yes, scala API accepts Array and assumes positional arguments, or Map and assumes named parameters



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425026420


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:
##########
@@ -903,4 +905,68 @@ class SparkSqlAstBuilder extends AstBuilder {
 
     (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER))
   }
+
+  /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {

Review Comment:
   the general principle is, if the parsed plan is in `ExecuteImmediateQuery`, the parser code should be in `AstBuilder` instead of `SparkSqlAstBuilder`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:
##########
@@ -903,4 +905,68 @@ class SparkSqlAstBuilder extends AstBuilder {
 
     (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER))
   }
+
+  /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {

Review Comment:
   the general principle is, if the parsed plan is in catalyst, the parser code should be in `AstBuilder` instead of `SparkSqlAstBuilder`



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425080254


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:
##########
@@ -903,4 +905,68 @@ class SparkSqlAstBuilder extends AstBuilder {
 
     (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER))
   }
+
+  /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {

Review Comment:
   Is your suggestion to move executeImmediate class to core ?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427057762


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:
##########
@@ -1465,6 +1465,25 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     assertAnalysisSuccess(finalPlan)
   }
 
+  test("Execute Immediate plan transformation") {
+    SimpleAnalyzer.catalogManager.tempVariableManager.create(
+        "res", "1", Literal(1), overrideIfExists = true)
+    SimpleAnalyzer.catalogManager.tempVariableManager.create(

Review Comment:
   let's clean up the created variable after this 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.

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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1431119683


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw QueryCompilationErrors.unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType)
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw QueryCompilationErrors.invalidQueryMixedQueryParameters()
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])

Review Comment:
   You can just point out the first one in the error message, but it is not critical.



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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #44093:
URL: https://github.com/apache/spark/pull/44093#issuecomment-1862781540

   I believe the issue has been solved already:
   ```
   ERROR:  Error installing bundler:
   	The last version of bundler (>= 0) to support your Ruby & RubyGems was 2.4.22. Try installing it with `gem install bundler -v 2.4.22`
   	bundler requires Ruby version >= 3.0.0. The current ruby version is 2.7.0.0.
   ```
   @milastdbx Could you rebase on the recent master, maybe need to rebase your master in fork (but not sure about that).


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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on PR #44093:
URL: https://github.com/apache/spark/pull/44093#issuecomment-1863042990

   The doc GA passed (others passed before): https://github.com/milastdbx/spark/actions/runs/7263315404/job/19788386092#logs
   
   +1, LGTM. Merging to master.
   Thank you, @milastdbx and @cloud-fan @srielau @dtenedor @beliefer for 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.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425042013


##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'

Review Comment:
   @srielau can you review the tests?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427047272


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(catalogManager.v1SessionCatalog.parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+      parser: ParserInterface,

Review Comment:
   nit: we don't need to pass it. We can get `catalogManager.v1SessionCatalog.parser` within this method



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425012167


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],

Review Comment:
   nit: 4 spaces indentation



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419141909


##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4:
##########
@@ -217,6 +217,7 @@ HOURS: 'HOURS';
 IDENTIFIER_KW: 'IDENTIFIER';
 IF: 'IF';
 IGNORE: 'IGNORE';
+IMMEDIATE: 'IMMEDIATE';

Review Comment:
   IMMEDIATE is not a reserved keyword, whereas EXECUTE is reserved in ANSI



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419125104


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   ```
   % find . -name "*.scala" -print0|xargs -0 grep -n 'string(visitStringLit(x))'
   ./sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:280:    ShowCatalogsCommand(Option(ctx.pattern).map(x => string(visitStringLit(x))))
   ./sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:845:    val path = Option(ctx.path).map(x => string(visitStringLit(x))).getOrElse("")
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:981:      Option(value).toSeq.map(x => key -> string(visitStringLit(x)))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:2879:      ctx.stringLit.asScala.map(x => string(visitStringLit(x))).mkString
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:3677:      Option(ctx.pattern).map(x => string(visitStringLit(x))))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:3815:      Option(value).toSeq.map(x => key -> string(visitStringLit(x)))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:3968:  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:4163:    ShowTables(ns, Option(ctx.pattern).map(x => string(visitStringLit(x))))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:4203:    ShowViews(ns, Option(ctx.pattern).map(x => string(visitStringLit(x))))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:4951:      Option(ctx.stringLit).map(x => string(visitStringLit(x))),
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:5054:    val pattern = Option(ctx.pattern).map(x => string(visitStringLit(x))).orElse(legacy.map(_.last))
   ```
   



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419125608


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   I meant getting string from StringLit node



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "dtenedor (via GitHub)" <gi...@apache.org>.
dtenedor commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1421247769


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1005,6 +1005,12 @@
     ],
     "sqlState" : "42702"
   },
+  "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : {
+    "message" : [
+      "Using statement contains multiple arguments with same alias (<aliases>)."

Review Comment:
   ```suggestion
         "The USING clause of this EXECUTE IMMEDIATE command contained multiple arguments with same alias (<aliases>), which is invalid; please update the command to specify unique aliases and then try it again."
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = extractQueryString(query)
+      val plan = CatalystSqlParser.parsePlan(queryString);
+
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect{ case n: NamedParameter => n })
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+          messageParameters = Map.empty)
+      } else {

Review Comment:
   you can drop the `else` since you raise an error in the previous case, de-denting the rest of the function.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))

Review Comment:
   these strings also appear on L65 above; please deduplicate into one place?



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.

Review Comment:
   please update the comments to complete sentences (each starting with a capital letter and ending with punctuation); here and elsewhere in the PR



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2338,6 +2356,12 @@
     ],
     "sqlState" : "42000"
   },
+  "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : {
+    "message" : [
+      "Variable type must be string type but got <varType>."

Review Comment:
   ```suggestion
         "Failed to run the EXECUTE IMMEDIATE command because the variable type must have string type, but instead it was <varType>."
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2261,6 +2273,12 @@
     },
     "sqlState" : "42000"
   },
+  "INVALID_STATEMENT_FOR_EXECUTE_INTO" : {
+    "message" : [
+      "The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement isn't: <sqlString>."

Review Comment:
   ```suggestion
         "Failed to run this EXECUTE IMMEDIATE command because the INTO clause is only valid for queries, but the given statement is not a query: <sqlString>."
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,

Review Comment:
   ```suggestion
       /**
         * We know that the expression is either UnresolvedAttribute or Alias,
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {

Review Comment:
   please update formatting to not leave a space before each `:` as specified by the Databricks Scala style guide: https://github.com/databricks/scala-style-guide



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v

Review Comment:
   please fix indentation (-2 spaces here and for the rest of the `match` block)



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425079513


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -557,7 +557,13 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
       notMatchedBySourceActions.toSeq)
   }
 
-  /**
+  override def visitMultipartIdentifierList(
+      ctx: MultipartIdentifierListContext): Seq[UnresolvedAttribute] = withOrigin(ctx) {

Review Comment:
   in USING we can either have literal or variable, and we need to distinguish between these two in SubstituteExecuteImmediate rule, hence i think its cleaner to have AST node than to pass Seq[String] directly and also I wouldn't be able to apply pattern matching



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427040236


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]])

Review Comment:
   nit: shall we use `Seq[UnresolvedAttribute]` directly? and Nil means no target variables.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427057040


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:
##########
@@ -1465,6 +1465,25 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     assertAnalysisSuccess(finalPlan)
   }
 
+  test("Execute Immediate plan transformation") {
+    SimpleAnalyzer.catalogManager.tempVariableManager.create(
+        "res", "1", Literal(1), overrideIfExists = true)

Review Comment:
   ```suggestion
         "res", "1", Literal(1), overrideIfExists = true)
   ```



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1428475360


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)

Review Comment:
   ```suggestion
           val plan = parseStatement(queryString, targetVariables)
   ```



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427817396


##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]

Review Comment:
   This is error returned by SET.
   I suggest we leave that for some other PR if improvement is needed there



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419270888


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate

Review Comment:
   to which generator did you write the tags like `@param`?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419119471


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1005,6 +1005,12 @@
     ],
     "sqlState" : "42702"
   },
+  "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : {
+    "message" : [
+      "Using statement contains multiple arguments with same alias (<aliases>)."
+    ],
+    "sqlState" : "42702"

Review Comment:
   Thinking about this... why is this a problem? It's a problem in INTO (for which we would use 42701



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419132503


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1005,6 +1005,12 @@
     ],
     "sqlState" : "42702"
   },
+  "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : {
+    "message" : [
+      "Using statement contains multiple arguments with same alias (<aliases>)."
+    ],
+    "sqlState" : "42702"

Review Comment:
   ```suggestion
       "sqlState" : "42701"
   ```
   
   It's really about duplicate assignments I think.



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {
+    "message" : [
+      "Parametrize query must either use positional, or named parameters, but not both."

Review Comment:
   ```suggestion
         "Parameterized query must either use positional, or named parameters, but not both."
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {
+    "message" : [
+      "Parametrize query must either use positional, or named parameters, but not both."
+    ],
+    "sqlState" : "42609"

Review Comment:
   42609 is for situations like ? = ? where Db2 can't to late binding because it can't derive the type from context.
   42613 seems most appropriate "clauses are mutually exclusive" (although parameter markers really aren't clauses...)



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2338,6 +2356,12 @@
     ],
     "sqlState" : "42000"
   },
+  "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : {
+    "message" : [
+      "Variable type must be \"STRING\" but got <varType>"

Review Comment:
   ```suggestion
         "Variable type must be string type but got <varType>"
   ```
   
   Thinking ahead at CHAR and VARCHAR...



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {

Review Comment:
   Isn't this a situation we already have from spark.sql("...", args)?



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2338,6 +2356,12 @@
     ],
     "sqlState" : "42000"
   },
+  "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : {
+    "message" : [
+      "Variable type must be \"STRING\" but got <varType>"

Review Comment:
   Isn't this just a generic DATATYPE_MISMATCH.NON_STRING_TYPE 42K09?
   Specifically if later we want to allow generic expressions, so it need not be a variable.



##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -230,6 +231,23 @@ statement
     | unsupportedHiveNativeCommands .*?                                #failNativeCommand
     ;
 
+executeImmediate
+    : EXECUTE IMMEDIATE queryParam=executeImmediateQueryParam (INTO LEFT_PAREN targetVariable=multipartIdentifierList RIGHT_PAREN)? (USING params=executeImmediateArgumentSeq)?

Review Comment:
   I just double checked that while Snowflake requires USING ( ), Google does has no (. ).
   Also PREPARE/EXECUTE does NOT use ( ).
   Can we make parens optional for USING and remove them for INTO (or also make them optional)?



##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4:
##########
@@ -217,6 +217,7 @@ HOURS: 'HOURS';
 IDENTIFIER_KW: 'IDENTIFIER';
 IF: 'IF';
 IGNORE: 'IGNORE';
+IMMEDIATE: 'IMMEDIATE';

Review Comment:
   It should be reserved keyword in  when we are strict ansi, but generally it shoudl be non-reserved.



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


Re: [PR] EXECUTE IMMEDIATE SQL support [spark]

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on PR #44093:
URL: https://github.com/apache/spark/pull/44093#issuecomment-1835143356

   Mind filing a JIRA 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.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425569964


##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]

Review Comment:
   Can we return a better error here?



##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+  "sqlState" : "07501",
+  "messageParameters" : {
+    "sqlString" : "INSERT INTO x VALUES (?)"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNRESOLVED_VARIABLE",
+  "sqlState" : "42883",
+  "messageParameters" : {
+    "searchPath" : "`SYSTEM`.`SESSION`",
+    "variableName" : "`id`"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where ? = id and :first = name' USING 1, 'name2' as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+  "sqlState" : "42613"
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELCT F(
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'''",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+a'
+
+EXECUTE IMMEDIATE b
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'a'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'na'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'('",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO a, b USING 10
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'name1'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 81,

Review Comment:
   We need to fix the context here. IIUC the '?' is typed as INT and thus id = ? needs to cast id as an INT and that is failing.
   A random user does not stand a chance here....



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }

Review Comment:
   If the statement uses named parameters and some USING expression is unnamed, then ignoring it would provide a poor experience for debugging.
   Note that we don't allow "null keys" for spark.sql() either.
   



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   The result set of a non-query is the "wild west". There is poor documentation or no documention.
   In the case of DML we must not support return of metadata as a result set in general because we would break future:
   SELECT FROM INSERT.
   Which is in the standard.
   
   This problem needs to be solved in general. Let's not make its resolution a precondition for this feature.   
    



##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+  "sqlState" : "07501",
+  "messageParameters" : {
+    "sqlString" : "INSERT INTO x VALUES (?)"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNRESOLVED_VARIABLE",
+  "sqlState" : "42883",
+  "messageParameters" : {
+    "searchPath" : "`SYSTEM`.`SESSION`",
+    "variableName" : "`id`"

Review Comment:
   Which id? There are two. If we had the fragment information, we would know it's about the USING part.



##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+NULL
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+1707
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.SparkNumberFormatException
+{
+  "errorClass" : "CAST_INVALID_INPUT",
+  "sqlState" : "22018",
+  "messageParameters" : {
+    "ansiConfig" : "\"spark.sql.ansi.enabled\"",
+    "expression" : "'invalid_cast_error_expected'",
+    "sourceType" : "\"STRING\"",
+    "targetType" : "\"INT\""
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "EXECUTE IMMEDIATE 'SELECT \\'invalid_cast_error_expected\\'' INTO res_id"
+  } ]
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+  "sqlState" : "07501",
+  "messageParameters" : {
+    "sqlString" : "INSERT INTO x VALUES (?)"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "UNRESOLVED_VARIABLE",
+  "sqlState" : "42883",
+  "messageParameters" : {
+    "searchPath" : "`SYSTEM`.`SESSION`",
+    "variableName" : "`id`"
+  }
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where ? = id and :first = name' USING 1, 'name2' as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+  "errorClass" : "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+  "sqlState" : "42613"
+}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELCT F(
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'''",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+a'
+
+EXECUTE IMMEDIATE b
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'a'",
+    "hint" : ""
+  }
+}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'na'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.catalyst.parser.ParseException
+{
+  "errorClass" : "PARSE_SYNTAX_ERROR",
+  "sqlState" : "42601",
+  "messageParameters" : {
+    "error" : "'('",
+    "hint" : ""
+  }
+}

Review Comment:
   Should we make this a feature not supported? We could allow expression and then catch it semanatically.



##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'

Review Comment:
   Done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425038124


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)

Review Comment:
   is it a user error? They can put an undefined variable in USING, 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.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425043738


##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id

Review Comment:
   does the target variable have to be pre-defined?



##########
sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out:
##########
@@ -0,0 +1,655 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, 'name1', named_struct('f1', 1, 's2', named_struct('f2', 101, 'f3', 'a'))),
+  (20, 'name2', named_struct('f1', 2, 's2', named_struct('f2', 202, 'f3', 'b'))),
+  (30, 'name3', named_struct('f1', 3, 's2', named_struct('f2', 303, 'f3', 'c'))),
+  (40, 'name4', named_struct('f1', 4, 's2', named_struct('f2', 404, 'f3', 'd'))),
+  (50, 'name5', named_struct('f1', 5, 's2', named_struct('f2', 505, 'f3', 'e'))),
+  (60, 'name6', named_struct('f1', 6, 's2', named_struct('f2', 606, 'f3', 'f'))),
+  (70, 'name7', named_struct('f1', 7, 's2', named_struct('f2', 707, 'f3', 'g')))
+AS tbl_view(id, name, data)
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+CREATE TABLE x (id INT) USING csv
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE sql_string STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true'
+-- !query schema
+struct<key:string,value:string>
+-- !query output
+spark.sql.ansi.enabled	true
+
+
+-- !query
+EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp'
+-- !query schema
+struct<id:int,name:string>
+-- !query output
+10	name1
+20	name2
+30	name3
+40	name4
+50	name5
+60	name6
+70	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'tbl_view_tmp' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int                 	                    
+name                	string
+
+
+-- !query
+EXECUTE IMMEDIATE 'DESCRIBE IDENTIFIER(:tblName)' USING 'x' as tblName
+-- !query schema
+struct<col_name:string,data_type:string,comment:string>
+-- !query output
+id                  	int
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\''
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE a STRING
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR a = 'name1'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+30	name3	{"f1":3,"s2":{"f2":303,"f3":"c"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2'
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2')
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+20	name2	{"f1":2,"s2":{"f2":202,"f3":"b"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT * from x
+-- !query schema
+struct<id:int>
+-- !query output
+1
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+DECLARE b INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SET VAR b = 40
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+40	name4	{"f1":4,"s2":{"f2":404,"f3":"d"}}
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>,p:string>
+-- !query output
+70	name7	{"f1":7,"s2":{"f2":707,"f3":"g"}}	name7
+
+
+-- !query
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT sql_string
+-- !query schema
+struct<sql_string:string>
+-- !query output
+SELECT id from tbl_view where name = :first
+
+
+-- !query
+DECLARE res_id INT
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+70
+
+
+-- !query
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+10
+
+
+-- !query
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT res_id
+-- !query schema
+struct<res_id:int>
+-- !query output
+42
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10
+-- !query schema
+struct<>
+-- !query output
+
+
+
+-- !query
+SELECT b, a
+-- !query schema
+struct<b:int,a:string>
+-- !query output
+10	name1
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a
+-- !query schema
+struct<id:int,name:string,data:struct<f1:int,s2:struct<f2:int,f3:string>>>
+-- !query output
+10	name1	{"f1":1,"s2":{"f2":101,"f3":"a"}}
+
+
+-- !query
+EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id

Review Comment:
   does the target variable have to be pre-defined?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425341642


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   we can also leave it for follow up PR



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   we can also leave it for follow up PR if needed



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1429112167


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))

Review Comment:
   Please, quote the data type by `toSQLType` (see other `AnalysisException` in `QueryCompilationErrors`, for instance):
   ```suggestion
               messageParameters = Map("varType" -> toSQLType(varReference.dataType)))
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.

Review Comment:
   ```suggestion
                 // that some rule does not accidentally resolve our parameters.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw invalidNameParameterizedQueryParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(aliases))
+          }
+        }
+
+        if (targetVariables.nonEmpty) {
+          SetVariable(targetVariables, queryPlan)
+        } else { queryPlan }
+    }
+
+  private def parseStatement(
+      queryString: String,
+      targetVariables: Seq[Expression]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    if (targetVariables.nonEmpty) {
+      try {
+        catalogManager.v1SessionCatalog.parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+          // Plan was sucessfully parsed, but query wasn't - throw.
+          throw new AnalysisException(
+            errorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+            messageParameters = Map("sqlString" -> queryString),

Review Comment:
   ```suggestion
               messageParameters = Map("sqlString" -> toSQLStmt(queryString)),
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw invalidNameParameterizedQueryParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(aliases))
+          }
+        }
+
+        if (targetVariables.nonEmpty) {
+          SetVariable(targetVariables, queryPlan)
+        } else { queryPlan }
+    }
+
+  private def parseStatement(
+      queryString: String,
+      targetVariables: Seq[Expression]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    if (targetVariables.nonEmpty) {
+      try {
+        catalogManager.v1SessionCatalog.parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+          // Plan was sucessfully parsed, but query wasn't - throw.

Review Comment:
   ```suggestion
             // Plan was successfully parsed, but query wasn't - throw.
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw invalidNameParameterizedQueryParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(aliases))
+          }
+        }
+
+        if (targetVariables.nonEmpty) {
+          SetVariable(targetVariables, queryPlan)
+        } else { queryPlan }
+    }
+
+  private def parseStatement(
+      queryString: String,
+      targetVariables: Seq[Expression]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    if (targetVariables.nonEmpty) {
+      try {
+        catalogManager.v1SessionCatalog.parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+          // Plan was sucessfully parsed, but query wasn't - throw.
+          throw new AnalysisException(
+            errorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+            messageParameters = Map("sqlString" -> queryString),
+            cause = Some(e))
+      }
+    } else {
+      val plan = catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+      // do not allow nested execute immediate
+      if (plan.containsPattern(EXECUTE_IMMEDIATE)) {
+        throw new AnalysisException(
+          errorClass = "NESTED_EXECUTE_IMMEDIATE",
+          messageParameters = Map("sqlString" -> queryString))

Review Comment:
   ```suggestion
             messageParameters = Map("sqlString" -> toSQLStmt(queryString)))
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw invalidNameParameterizedQueryParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.

Review Comment:
   ```suggestion
                 // that some rule does not accidentally resolve our parameters.
   ```



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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1431119683


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw QueryCompilationErrors.unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType)
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw QueryCompilationErrors.invalidQueryMixedQueryParameters()
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])

Review Comment:
   You can just point out the first one in the error message.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1418979049


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable

Review Comment:
   ```suggestion
    * @param args Expressions to be replaced in place of parameters of queryText/queryVariable
   ```



##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -72,6 +72,7 @@ singleTableSchema
 
 statement
     : query                                                            #statementDefault
+    | executeImmediate                                                 # visitExecuteImmediate

Review Comment:
   minor but just for consistency:
   ```suggestion
       | executeImmediate                                                 #visitExecuteImmediate
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = extractQueryString(query)
+      val plan = CatalystSqlParser.parsePlan(queryString);
+
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect{ case n: NamedParameter => n })
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_PARAMETRIZED_QUERY",
+          messageParameters = Map.empty)
+      } else {
+        // parser does not distinguish between variables and columns.
+        // Make sure that we have either literal or variable in expressions.
+
+        if (posNodes.nonEmpty) {
+          // Add aggregation or a project.
+          PosParameterizedQuery(
+            plan,
+            // we need to resolve arguments before Resolution batch to make sure
+            // that some rule does not accidently resolve our parameters.
+            // we do not want this as they can resolve some unsupported parameters
+            resolveArguments(expressions))
+        } else {
+          val namedExpressions = expressions.collect {
+            case (e: NamedExpression) => e
+          }
+
+          NameParameterizedQuery(
+            plan,
+            namedExpressions.map(_.name),
+            // we need to resolve arguments before Resolution batch to make sure
+            // that some rule does not accidently resolve our parameters.
+            // we do not want this as they can resolve some unsupported parameters

Review Comment:
   ditto



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -166,6 +167,52 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     query.optionalMap(ctx.ctes)(withCTE)
   }
 
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {
+    // because of parsing rules, we know that either queryParam or targetVariable is set
+    // hence use Either to represent this
+    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(stringLitToStr(sl)))
+    val queryVariable = Option(ctx.queryParam.multipartIdentifier)
+      .map(mpi => Right(UnresolvedAttribute(visitMultipartIdentifier(mpi))))
+
+    val targetVars = Option(ctx.targetVariable)
+      .map(v => visitMultipartIdentifierList(v))
+    val exprs = visitExecuteImmediateArgumentSeq(ctx.params)
+
+    validateExecImmediateArguments(exprs, ctx.params)
+    ExecuteImmediateQuery(exprs, queryString.getOrElse(queryVariable.get), targetVars)
+  }
+
+  private def validateExecImmediateArguments(
+    expressions: Seq[Expression],
+    ctx : ExecuteImmediateArgumentSeqContext) : Unit = {
+    val duplicateAliases = expressions
+      .filter {
+        case a: Alias => true
+        case _ => false

Review Comment:
   Just use `isInstanceOf` like `.filter(_.isInstanceOf[Alias])`



##########
docs/sql-ref-ansi-compliance.md:
##########
@@ -9,9 +9,9 @@ license: |
   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
- 

Review Comment:
   The changes are unnecessary, this might just cause conflicts. You can open a separate PR and fix the issue in all *.md.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   > this logic is used in couple of place
   
   Where?
   ```
   $ find . -name "*.scala" -print0|xargs -0 grep -n 'stringLitToStr'
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:173:    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(stringLitToStr(sl)))
   ./sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:3968:  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2261,6 +2273,12 @@
     },
     "sqlState" : "42000"
   },
+  "INVALID_STATEMENT_FOR_EXECUTE_INTO" : {
+    "message" : [
+      "The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement isn't: <sqlString> ."

Review Comment:
   ```suggestion
         "The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement isn't: <sqlString>."
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = extractQueryString(query)
+      val plan = CatalystSqlParser.parsePlan(queryString);
+
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect{ case n: NamedParameter => n })
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_PARAMETRIZED_QUERY",
+          messageParameters = Map.empty)
+      } else {
+        // parser does not distinguish between variables and columns.
+        // Make sure that we have either literal or variable in expressions.
+
+        if (posNodes.nonEmpty) {
+          // Add aggregation or a project.
+          PosParameterizedQuery(
+            plan,
+            // we need to resolve arguments before Resolution batch to make sure
+            // that some rule does not accidently resolve our parameters.
+            // we do not want this as they can resolve some unsupported parameters

Review Comment:
   ```suggestion
               // We do not want this as they can resolve some unsupported parameters.
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala:
##########
@@ -32,7 +32,22 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL
     intercept[ParseException](sql(sqlText).collect())
   }
 
-  test("NAMED_PARAMETER_SUPPORT_DISABLED: named arguments not turned on") {
+  test("EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES: duplicate aliases provided in using statement") {
+    checkError(
+      exception = parseException(
+        "EXECUTE IMMEDIATE 'SELECT 1707 WHERE ? = 1' USING 1 as first" +
+          ", 2 as first, 3 as second, 4 as second, 5 as third"),
+      errorClass = "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES",
+      parameters = Map("aliases"-> "`second`, `first`"),
+      context = ExpectedContext(
+        "EXECUTE IMMEDIATE 'SELECT 1707 WHERE ? = 1' USING 1 as first" +

Review Comment:
   Minor but is the string same as above? If so, put the string to a val.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate

Review Comment:
   Doesn't this confuses the doc generator since the class doesn't have such parameters.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate
+ * @param targetVariable Variable to store result into if specified
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = extractQueryString(query)
+      val plan = CatalystSqlParser.parsePlan(queryString);
+
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect{ case n: NamedParameter => n })
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_PARAMETRIZED_QUERY",
+          messageParameters = Map.empty)
+      } else {
+        // parser does not distinguish between variables and columns.
+        // Make sure that we have either literal or variable in expressions.
+
+        if (posNodes.nonEmpty) {
+          // Add aggregation or a project.
+          PosParameterizedQuery(
+            plan,
+            // we need to resolve arguments before Resolution batch to make sure

Review Comment:
   ```suggestion
               // We need to resolve arguments before Resolution batch to make sure
   ```



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {

Review Comment:
   This seems generic name but the error is pretty specific. Please, reformulate the error class name. 



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425024729


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -557,7 +557,13 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
       notMatchedBySourceActions.toSeq)
   }
 
-  /**
+  override def visitMultipartIdentifierList(
+      ctx: MultipartIdentifierListContext): Seq[UnresolvedAttribute] = withOrigin(ctx) {

Review Comment:
   Since we have dedicated code to resolve variable in EXECUTE IMMEDIATE, I think we can use `Seq[String]` directly instead of `UnresolvedAttribute`



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425084070


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)

Review Comment:
   parser needs to pass either Alias on UnresolvedAttribute here, so if we got here somehow, its a bug



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1415300262


##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4:
##########
@@ -217,6 +217,7 @@ HOURS: 'HOURS';
 IDENTIFIER_KW: 'IDENTIFIER';
 IF: 'IF';
 IGNORE: 'IGNORE';
+IMMEDIATE: 'IMMEDIATE';

Review Comment:
   it seems to be a reserved keyword in standard



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427055448


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala:
##########
@@ -785,6 +785,41 @@ class AnalysisErrorSuite extends AnalysisTest {
        "filter (where nth_value(e, 2) over(order by b) > 1) FROM TaBlE2"),
     "FILTER expression contains window function" :: Nil)
 
+  errorClassTest(
+    "EXEC IMMEDIATE - both positional and named used",
+    CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'SELECT 42 where ? = :first'" +
+      " USING 1, 2 as first"),
+    "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+    Map.empty);
+
+  test("EXEC IMMEDIATE - non string variable as sqlString parameter") {
+    var execImmediatePlan = ExecuteImmediateQuery(
+      Seq.empty,
+      scala.util.Right(UnresolvedAttribute("testVarA")),

Review Comment:
   where is `testVarA` defined?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427956668


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -558,6 +560,73 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
   }
 
   /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {
+    // Because of how parsing rules are written, we know that either
+    // queryParam or targetVariable is non null - hence use Either to represent this.
+    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(string(visitStringLit(sl))))

Review Comment:
   What do you mean ?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427812944


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala:
##########
@@ -785,6 +785,41 @@ class AnalysisErrorSuite extends AnalysisTest {
        "filter (where nth_value(e, 2) over(order by b) > 1) FROM TaBlE2"),
     "FILTER expression contains window function" :: Nil)
 
+  errorClassTest(
+    "EXEC IMMEDIATE - both positional and named used",
+    CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'SELECT 42 where ? = :first'" +
+      " USING 1, 2 as first"),
+    "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+    Map.empty);
+
+  test("EXEC IMMEDIATE - non string variable as sqlString parameter") {
+    var execImmediatePlan = ExecuteImmediateQuery(
+      Seq.empty,
+      scala.util.Right(UnresolvedAttribute("testVarA")),

Review Comment:
   in `AnalysisTest.scala`



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425039116


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    targetVariables.map { expressions =>

Review Comment:
   where do we assign the result to the variable?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425031603


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }

Review Comment:
   shall we fail or ignore un-named parameters? cc @srielau 



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419280588


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {

Review Comment:
   No, we don't because there is not method in Scala API which accepts both named and positional parameters.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1424029478


##########
sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql:
##########
@@ -0,0 +1,96 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))),
+  (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))),
+  (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))),
+  (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))),
+  (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))),
+  (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))),
+  (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g")))
+AS tbl_view(id, name, data);
+CREATE TABLE x (id INT) USING csv;
+
+DECLARE sql_string STRING;
+SET VAR sql_string = 'SELECT * from tbl_view where name = "name1"';
+
+-- test execute immediate without parameters
+EXECUTE IMMEDIATE sql_string;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = "name1"';
+
+-- test positional paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?';
+DECLARE a STRING;
+SET VAR a = "name1";
+EXECUTE IMMEDIATE sql_string USING "name1", "name3";
+EXECUTE IMMEDIATE sql_string USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING "name1", "name3";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, "name2");
+-- test positonal command
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1;
+SELECT * from x;
+
+-- test named parameters paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+DECLARE b INT;
+SET VAR b = 40;
+EXECUTE IMMEDIATE sql_string USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE sql_string USING b as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING "name7" as first, b as second;
+-- named parameter used multiple times
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING "name7" as first;
+
+-- test named command and setup for next test
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first';
+SELECT sql_string;
+
+-- test into
+DECLARE res_id INT;
+EXECUTE IMMEDIATE sql_string INTO res_id USING "name7" as first;
+SELECT res_id;
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first;
+SELECT res_id;
+
+-- test into without using
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id;
+SELECT res_id;
+
+-- multiple INTOs
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10;
+SELECT b, a;
+
+-- use AS for using positional params
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a;
+
+-- test errors

Review Comment:
   Fixed



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425074544


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   I'll let @srielau  comment



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425039116


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    targetVariables.map { expressions =>

Review Comment:
   where do we assign the result to the variable?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425020470


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   non-query command can also return value, e.g. `SHOW TABLE`, and DML commands can also return value like number of rows added/deleted. Why do we have this limitation? cc @srielau as well



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419119471


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1005,6 +1005,12 @@
     ],
     "sqlState" : "42702"
   },
+  "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES" : {
+    "message" : [
+      "Using statement contains multiple arguments with same alias (<aliases>)."
+    ],
+    "sqlState" : "42702"

Review Comment:
   Thinking about this... why is this a problem? It's a problem in INTO (for which we would use 42701



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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #44093: [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support
URL: https://github.com/apache/spark/pull/44093


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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1429943238


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)

Review Comment:
   It seems not resolved.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan =
+          parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])
+
+            if (nonAliases.nonEmpty) {
+              throw invalidNameParameterizedQueryParametersMustBeNamed(nonAliases)
+            }
+
+            NameParameterizedQuery(
+              plan,
+              aliases.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(aliases))
+          }
+        }
+
+        if (targetVariables.nonEmpty) {
+          SetVariable(targetVariables, queryPlan)
+        } else { queryPlan }
+    }
+
+  private def parseStatement(
+      queryString: String,
+      targetVariables: Seq[Expression]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    if (targetVariables.nonEmpty) {
+      try {
+        catalogManager.v1SessionCatalog.parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          catalogManager.v1SessionCatalog.parser.parsePlan(queryString)
+
+          // Plan was sucessfully parsed, but query wasn't - throw.

Review Comment:
   It seems not resolved.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,198 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.{invalidNameParameterizedQueryParametersMustBeNamed, unresolvedVariableError, unsupportedParameterExpression}
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query

Review Comment:
   ```suggestion
    * @param args parameters of query
    * @param query query string or variable
    * @param targetVariables variables to store the result of the query
   ```



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425028072


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.
+    // Otherwise, it can be anything.
+    targetVariables.map { expressions =>
+      try {
+        parser.parseQuery(queryString)
+      } catch {
+        case e: ParseException =>
+          // Since we do not have a way of telling that parseQuery failed because of
+          // actual parsing error or because statement was passed where query was expected,
+          // we need to make sure that parsePlan wouldn't throw
+          parser.parsePlan(queryString)
+
+          // Plan was sucessfully parsed, but query wasn't - throw.
+          throw new AnalysisException(
+            errorClass = "INVALID_STATEMENT_FOR_EXECUTE_INTO",
+            messageParameters = Map("sqlString" -> queryString),
+            cause = Some(e))
+      }
+    }.getOrElse { parser.parsePlan(queryString) }

Review Comment:
   we should ban nested Execute Immediate



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427880560


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }

Review Comment:
   we are allowing empty keys tho.
   <img width="783" alt="image" src="https://github.com/apache/spark/assets/150366084/2b7271b7-1c25-4f1f-9320-15fa0f5568e9">
   



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


Re: [PR] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1413975389


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   this logic is used in couple of places, i think it makes sense to isolate it in seperate method



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425017670


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,

Review Comment:
   nit: 4 spaces indentation



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419120219


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args Expressions to be replaced inplace of parameters of queryText/queryVariable
+ * @param queryText Query text as string literal as an option
+ * @param queryVariable Unresolved query attribute used as query in execute immediate

Review Comment:
   which doc generator?



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419292326


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   why not have it here for future uses ? 
   
   I mean I can refactor as well, but I see benefit of having it so it is reused from now on



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419277785


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -3909,6 +3943,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     }
   }
 
+  /**
+   * Converts a string literal to a string.
+   */
+  protected def stringLitToStr(x: StringLitContext) = string(visitStringLit(x))

Review Comment:
   If you don't plan to refactor the code, and re-use your new method, the method is redundant, and should be inlined.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419247729


##########
sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4:
##########
@@ -230,6 +231,23 @@ statement
     | unsupportedHiveNativeCommands .*?                                #failNativeCommand
     ;
 
+executeImmediate
+    : EXECUTE IMMEDIATE queryParam=executeImmediateQueryParam (INTO LEFT_PAREN targetVariable=multipartIdentifierList RIGHT_PAREN)? (USING params=executeImmediateArgumentSeq)?

Review Comment:
   using parens are optional and into changed to without parens



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1419295861


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2085,6 +2091,12 @@
     },
     "sqlState" : "22023"
   },
+  "INVALID_PARAMETRIZED_QUERY" : {

Review Comment:
   So error returned would be UNBOUND_SQL_PARAMETER



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425342125


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }

Review Comment:
   My take here is that we should ignore unnamed parameters



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1428713899


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -558,6 +560,73 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
   }
 
   /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {
+    // Because of how parsing rules are written, we know that either
+    // queryParam or targetVariable is non null - hence use Either to represent this.
+    val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(string(visitStringLit(sl))))

Review Comment:
   We need to invoke the parser here anyway if the query string is a session variable. The code will be more convoluted if we parse the query string in two different places.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425081840


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala:
##########
@@ -903,4 +905,68 @@ class SparkSqlAstBuilder extends AstBuilder {
 
     (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER))
   }
+
+  /**
+   * Returns the parameters for [[ExecuteImmediateQuery]] logical plan.
+   * Expected format:
+   * {{{
+   *   EXECUTE IMMEDIATE {query_string|string_literal}
+   *   [INTO target1, target2] [USING param1, param2, ...]
+   * }}}
+   */
+  override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) {

Review Comment:
   I'm suggesting to move the parser code



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1425021485


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.{ParseException, ParserInterface}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args
+ *   parameters of query
+ * @param query
+ *   query string or variable
+ * @param targetVariables
+ *   variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+  args: Seq[Expression],
+  query: Either[String, UnresolvedAttribute],
+  targetVariables: Option[Seq[UnresolvedAttribute]],
+  parser: ParserInterface)
+    extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan]
+    with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other =>
+        throw SparkException.internalError(
+          "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw new AnalysisException(
+            errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+            messageParameters = Map("varType" -> varReference.dataType.simpleString))
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariablesOpt, parser) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(parser, queryString, targetVariablesOpt)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw new AnalysisException(
+            errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+            messageParameters = Map.empty)
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val namedExpressions = expressions.collect { case (e: NamedExpression) => e }
+
+            NameParameterizedQuery(
+              plan,
+              namedExpressions.map(_.name),
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidently resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters.
+              resolveArguments(namedExpressions))
+          }
+        }
+
+        targetVariablesOpt
+          .map(variables => {
+            SetVariable(variables, queryPlan)
+          })
+          .getOrElse { queryPlan }
+    }
+
+  private def parseStatement(
+    parser: ParserInterface,
+    queryString: String,
+    targetVariables: Option[Seq[Expression]]): LogicalPlan = {
+    // If targetVariables is defined, statement needs to be a query.

Review Comment:
   I think what matters is the logical plan schema which needs to match the target variables, and the plan can be anything.



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1427051770


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -58,7 +59,8 @@ import org.apache.spark.util.random.RandomSampler
  * The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
  * TableIdentifier.
  */
-class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
+class AstBuilder
+    extends DataTypeAstBuilder with SQLConfHelper with Logging {

Review Comment:
   unnecessary change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1429065647


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1975,6 +1981,12 @@
     },
     "sqlState" : "42K0E"
   },
+  "INVALID_NAME_PARAMETERIZED_QUERY_ALL_PARAMETERS_MUST_BE_NAMED" : {

Review Comment:
   This is too long, please, make it shorter. For instance: `MISSED_PARAMETER_NAME`



##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala:
##########
@@ -32,6 +32,33 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL
     intercept[ParseException](sql(sqlText).collect())
   }
 
+  test("EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES: duplicate aliases provided in using statement") {
+    val query = "EXECUTE IMMEDIATE 'SELECT 1707 WHERE ? = 1' USING 1 as first" +
+      ", 2 as first, 3 as second, 4 as second, 5 as third"
+    checkError(
+      exception = parseException(query),
+      errorClass = "EXEC_IMMEDIATE_DUPLICATE_ARGUMENT_ALIASES",
+      parameters = Map("aliases" -> "`second`, `first`"),
+      context = ExpectedContext(
+        query,
+        start = 0,
+        stop = 109)
+    )
+  }
+
+  test("PARSE_SYNTAX_ERROR: Execute immediate syntax error with INTO specified") {
+    val query = "EXECUTE IMMEDIATE 'SELCT 1707 WHERE ? = 1' INTO a USING 1"
+    checkError(
+      exception = parseException(query),
+      errorClass = "PARSE_SYNTAX_ERROR",
+      parameters = Map("error" -> "'SELCT'", "hint" -> ""),
+      context = ExpectedContext(
+        start = 0,
+        stop = 56,
+        fragment = "EXECUTE IMMEDIATE 'SELCT 1707 WHERE ? = 1' INTO a USING 1")

Review Comment:
   Seems like it can be substituted by `query`, can't be?



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2157,6 +2169,12 @@
     ],
     "sqlState" : "42602"
   },
+  "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT" : {

Review Comment:
   Please, make it shorter, like `MIXED_QUERY_PARAMETERS`



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


Re: [PR] [SPARK-46246][SQL] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1431093953


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Seq[UnresolvedAttribute])
+  extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+  extends Rule[LogicalPlan]
+  with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression): Expression = {
+
+    /**
+     * We know that the expression is either UnresolvedAttribute or Alias, as passed from the
+     * parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If it
+     * is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        getVariableReference(u, u.nameParts)
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+      case other =>
+        throw QueryCompilationErrors.unsupportedParameterExpression(other)
+    }
+  }
+
+  def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either: Either[String, UnresolvedAttribute]): String = {
+    either match {
+      case Left(v) => v
+      case Right(u) =>
+        val varReference = getVariableReference(u, u.nameParts)
+
+        if (!varReference.dataType.sameType(StringType)) {
+          throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType)
+        }
+
+        // Call eval with null value passed instead of a row.
+        // This is ok as this is variable and invoking eval should
+        // be independent of row value.
+        varReference.eval(null).toString
+    }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan =
+    plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+      case ExecuteImmediateQuery(expressions, query, targetVariables) =>
+        val queryString = extractQueryString(query)
+        val plan = parseStatement(queryString, targetVariables)
+
+        val posNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+        }.flatten
+        val namedNodes = plan.collect { case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: NamedParameter => n })
+        }.flatten
+
+        val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+          plan
+        } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+          throw QueryCompilationErrors.invalidQueryMixedQueryParameters()
+        } else {
+          if (posNodes.nonEmpty) {
+            PosParameterizedQuery(
+              plan,
+              // We need to resolve arguments before Resolution batch to make sure
+              // that some rule does not accidentally resolve our parameters.
+              // We do not want this as they can resolve some unsupported parameters
+              resolveArguments(expressions))
+          } else {
+            val aliases = expressions.collect {
+              case (e: Alias) => e
+            }
+            val nonAliases = expressions.filter(!_.isInstanceOf[Alias])

Review Comment:
   I need to collect them for error mesage



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "srielau (via GitHub)" <gi...@apache.org>.
srielau commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1420801268


##########
sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql:
##########
@@ -0,0 +1,96 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))),
+  (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))),
+  (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))),
+  (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))),
+  (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))),
+  (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))),
+  (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g")))
+AS tbl_view(id, name, data);
+CREATE TABLE x (id INT) USING csv;
+
+DECLARE sql_string STRING;
+SET VAR sql_string = 'SELECT * from tbl_view where name = "name1"';

Review Comment:
   I understand the temptation, but better stay clear of double quotes. We have a config to switch them to identifiers, An dI have not given up the dream to change the default....)



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2261,6 +2273,12 @@
     },
     "sqlState" : "42000"
   },
+  "INVALID_STATEMENT_FOR_EXECUTE_INTO" : {
+    "message" : [
+      "The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement isn't: <sqlString>."
+    ],
+    "sqlState" : "42606"

Review Comment:
   "An invalid hexadecimal constant has been detected"?
   How about:
   "The option specified on PREPARE or EXECUTE is not valid."
   (07 is all about dynamic SQL)
   ```suggestion
       "sqlState" : "07501"
   ```



##########
docs/sql-ref-syntax-aux-exec-imm.md:
##########
@@ -0,0 +1,86 @@
+---
+layout: global
+title: EXECUTE IMMEDIATE
+displayTitle: EXECUTE IMMEDIATE
+license: |
+  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.
+---
+
+### Description
+
+Executes a sql statement provided as a `STRING`, optionally passing `arg_exprN` to parameter markers and assigning the results to `var_nameN`.
+
+### Syntax
+
+```sql
+EXECUTE IMMEDIATE sql_string
+        [ INTO (var_name [, …] ) ]
+        [ USING ( arg_expr [ AS ] [alias] [, …] ) ]
+```
+
+### Parameters
+
+* **sql_string**
+
+  A STRING expression producing a well-formed SQL statement.
+
+* **INTO ( var_name [, …] )**
+
+    Optionally returns the results of a single row query into SQL variables.
+    If the query returns no rows the result is NULL.
+    - `var_name`
+    A SQL variable. A variable may not be referenced more than once.
+
+* **USING (arg_expr [, …] )**
+
+  Optionally, if sql_string contains parameter markers, binds in values to the parameters.
+  - `arg_expr`
+  An expression that binds to a parameter marker.
+  If the parameter markers are unnamed the binding is by position.
+  For unnamed parameter markers, binding is by name.
+  - `alias`
+    Overrides the name used to bind `arg_expr` to a named parameter marker
+
+  Each named parameter marker must be matched once. Not all arg_expr must be matched.
+
+
+### Examples
+
+```sql
+-- A self-contained execution using a literal string
+EXECUTE IMMEDIATE ‘SELECT SUM(c1) FROM VALUES(?), (?)’ USING (5, 6);

Review Comment:
   And the example need to be fixed. Best lead with USING without parens, since that's closer to the standard.
   Also, you need to fix the single quotes (copy post-problem from docs). '
   ```suggestion
   EXECUTE IMMEDIATE 'SELECT SUM(c1) FROM VALUES(?), (?)' USING 5, 6;
   ```



##########
sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql:
##########
@@ -0,0 +1,96 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))),
+  (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))),
+  (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))),
+  (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))),
+  (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))),
+  (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))),
+  (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g")))
+AS tbl_view(id, name, data);
+CREATE TABLE x (id INT) USING csv;
+
+DECLARE sql_string STRING;
+SET VAR sql_string = 'SELECT * from tbl_view where name = "name1"';
+
+-- test execute immediate without parameters
+EXECUTE IMMEDIATE sql_string;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = "name1"';
+
+-- test positional paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?';
+DECLARE a STRING;
+SET VAR a = "name1";
+EXECUTE IMMEDIATE sql_string USING "name1", "name3";
+EXECUTE IMMEDIATE sql_string USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING "name1", "name3";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, "name2");
+-- test positonal command
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1;
+SELECT * from x;
+
+-- test named parameters paramete

Review Comment:
   ```suggestion
   -- test named parameters
   ```



##########
docs/sql-ref-syntax-aux-exec-imm.md:
##########
@@ -0,0 +1,86 @@
+---
+layout: global
+title: EXECUTE IMMEDIATE
+displayTitle: EXECUTE IMMEDIATE
+license: |
+  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.
+---
+
+### Description
+
+Executes a sql statement provided as a `STRING`, optionally passing `arg_exprN` to parameter markers and assigning the results to `var_nameN`.
+
+### Syntax
+
+```sql
+EXECUTE IMMEDIATE sql_string
+        [ INTO (var_name [, …] ) ]

Review Comment:
   Since we changed the grammar we need to fix the docs here



##########
sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql:
##########
@@ -0,0 +1,96 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))),
+  (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))),
+  (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))),
+  (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))),
+  (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))),
+  (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))),
+  (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g")))
+AS tbl_view(id, name, data);
+CREATE TABLE x (id INT) USING csv;
+
+DECLARE sql_string STRING;
+SET VAR sql_string = 'SELECT * from tbl_view where name = "name1"';
+
+-- test execute immediate without parameters
+EXECUTE IMMEDIATE sql_string;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = "name1"';
+
+-- test positional paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?';
+DECLARE a STRING;
+SET VAR a = "name1";
+EXECUTE IMMEDIATE sql_string USING "name1", "name3";
+EXECUTE IMMEDIATE sql_string USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING "name1", "name3";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, "name2");
+-- test positonal command
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1;
+SELECT * from x;
+
+-- test named parameters paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+DECLARE b INT;
+SET VAR b = 40;
+EXECUTE IMMEDIATE sql_string USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE sql_string USING b as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING "name7" as first, b as second;
+-- named parameter used multiple times
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING "name7" as first;
+
+-- test named command and setup for next test
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first';
+SELECT sql_string;
+
+-- test into
+DECLARE res_id INT;
+EXECUTE IMMEDIATE sql_string INTO res_id USING "name7" as first;
+SELECT res_id;
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first;
+SELECT res_id;
+
+-- test into without using
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id;
+SELECT res_id;
+
+-- multiple INTOs
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10;
+SELECT b, a;
+
+-- use AS for using positional params
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a;
+
+-- test errors

Review Comment:
   I don't see tests for:
   * NULL on empty query
   * error on too many rows
   * error on mismatch between INTO and SELECT list cardinality
   * Testing on implcicit casting between SELECT list and INTO, as well as errors if not possible.
   * Duplicate alias in USING.
   * Duplicate INTO entry
   * We should have a wider variety of statement types. E.g. CREATE, DESCRIBE, SET (config) 
   
   
   
   



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1422220330


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.catalyst.analysis
+
+import scala.util.{Either, Left, Right}
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SetVariable}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, EXECUTE_IMMEDIATE, TreePattern}
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError
+import org.apache.spark.sql.types.StringType
+
+/**
+ * Logical plan representing execute immediate query.
+ *
+ * @param args parameters of query
+ * @param query query string or variable
+ * @param targetVariables variables to store the result of the query
+ */
+case class ExecuteImmediateQuery(
+    args: Seq[Expression],
+    query: Either[String, UnresolvedAttribute],
+    targetVariables: Option[Seq[UnresolvedAttribute]]) extends UnresolvedLeafNode {
+  final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE)
+}
+
+/**
+ * This rule substitutes execute immediate query node with plan that is passed as string literal
+ * or session parameter.
+ */
+class SubstituteExecuteImmediate(val catalogManager: CatalogManager)
+    extends Rule[LogicalPlan] with ColumnResolutionHelper {
+
+  def resolveVariable(e: Expression) : Expression = {
+    /* We know that the expression is either UnresolvedAttribute or Alias,
+     * as passed from the parser.
+     * If it is an UnresolvedAttribute, we look it up in the catalog and return it.
+     * If it is an Alias, we resolve the child and return an Alias with the same name.
+     */
+    e match {
+      case u: UnresolvedAttribute =>
+        lookupVariable(u.nameParts) match {
+          case Some(variable) =>
+            variable.copy(canFold = false)
+          case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+        }
+
+      case a: Alias =>
+        Alias(resolveVariable(a.child), a.name)()
+
+      case other => throw SparkException.internalError(
+        "Unexpected variable expression in ParametrizedQuery: " + other)
+    }
+  }
+
+  def resolveArguments(expressions : Seq[Expression]): Seq[Expression] = {
+    expressions.map { exp =>
+      if (exp.resolved) {
+        exp
+      } else {
+        resolveVariable(exp)
+      }
+    }
+  }
+
+  def extractQueryString(either : Either[String, UnresolvedAttribute]) : String = {
+    either match {
+        case Left(v) => v
+        case Right(u) =>
+          val varReference = lookupVariable(u.nameParts) match {
+            case Some(variable) => variable.copy(canFold = false)
+            case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION"))
+          }
+
+          if (!varReference.dataType.sameType(StringType)) {
+            throw new AnalysisException(
+              errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE",
+              messageParameters = Map(
+                "varType" -> varReference.dataType.simpleString))
+          }
+
+          // call eval with null row.
+          // this is ok as this is variable and invoking eval should
+          // be independent of row
+          varReference.eval(null).toString
+      }
+  }
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(
+    _.containsPattern(EXECUTE_IMMEDIATE), ruleId) {
+    case ExecuteImmediateQuery(expressions, query, targetVariablesOpt) =>
+      val queryString = extractQueryString(query)
+      val plan = CatalystSqlParser.parsePlan(queryString);
+
+      val posNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect { case n: PosParameter => n })
+      }.flatten
+      val namedNodes = plan.collect {
+        case p: LogicalPlan =>
+          p.expressions.flatMap(_.collect{ case n: NamedParameter => n })
+      }.flatten
+
+      val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) {
+        plan
+      } else if (posNodes.nonEmpty && namedNodes.nonEmpty) {
+        throw new AnalysisException(
+          errorClass = "INVALID_QUERY_BOTH_POSITIONAL_AND_NAMED_PARAMETERS_PRESENT",
+          messageParameters = Map.empty)
+      } else {

Review Comment:
   this whole if expression gets assigned to a query plan variable so i cannot drop else



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


Re: [PR] [SPARK-46246] EXECUTE IMMEDIATE SQL support [spark]

Posted by "milastdbx (via GitHub)" <gi...@apache.org>.
milastdbx commented on code in PR #44093:
URL: https://github.com/apache/spark/pull/44093#discussion_r1422679449


##########
sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql:
##########
@@ -0,0 +1,96 @@
+-- Automatically generated by SQLQueryTestSuite
+-- !query
+CREATE TEMPORARY VIEW tbl_view AS SELECT * FROM VALUES
+  (10, "name1", named_struct("f1", 1, "s2", named_struct("f2", 101, "f3", "a"))),
+  (20, "name2", named_struct("f1", 2, "s2", named_struct("f2", 202, "f3", "b"))),
+  (30, "name3", named_struct("f1", 3, "s2", named_struct("f2", 303, "f3", "c"))),
+  (40, "name4", named_struct("f1", 4, "s2", named_struct("f2", 404, "f3", "d"))),
+  (50, "name5", named_struct("f1", 5, "s2", named_struct("f2", 505, "f3", "e"))),
+  (60, "name6", named_struct("f1", 6, "s2", named_struct("f2", 606, "f3", "f"))),
+  (70, "name7", named_struct("f1", 7, "s2", named_struct("f2", 707, "f3", "g")))
+AS tbl_view(id, name, data);
+CREATE TABLE x (id INT) USING csv;
+
+DECLARE sql_string STRING;
+SET VAR sql_string = 'SELECT * from tbl_view where name = "name1"';
+
+-- test execute immediate without parameters
+EXECUTE IMMEDIATE sql_string;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = "name1"';
+
+-- test positional paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = ? or name = ?';
+DECLARE a STRING;
+SET VAR a = "name1";
+EXECUTE IMMEDIATE sql_string USING "name1", "name3";
+EXECUTE IMMEDIATE sql_string USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING "name1", "name3";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, "name2";
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, "name2");
+-- test positonal command
+EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1;
+SELECT * from x;
+
+-- test named parameters paramete
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+DECLARE b INT;
+SET VAR b = 40;
+EXECUTE IMMEDIATE sql_string USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE sql_string USING b as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, "name7" as first;
+EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING "name7" as first, b as second;
+-- named parameter used multiple times
+EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING "name7" as first;
+
+-- test named command and setup for next test
+EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first';
+SELECT sql_string;
+
+-- test into
+DECLARE res_id INT;
+EXECUTE IMMEDIATE sql_string INTO res_id USING "name7" as first;
+SELECT res_id;
+EXECUTE IMMEDIATE sql_string INTO res_id USING a as first;
+SELECT res_id;
+
+-- test into without using
+SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second';
+EXECUTE IMMEDIATE 'SELECT 42' INTO res_id;
+SELECT res_id;
+
+-- multiple INTOs
+EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10;
+SELECT b, a;
+
+-- use AS for using positional params
+EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a;
+
+-- test errors

Review Comment:
   Updates tests for everything suggested except
   `We should have a wider variety of statement types. E.g. CREATE, DESCRIBE, SET (config)`
   
   These statements are defined in SparkSqlParser which is defined in sql/core project.
   
   sql/core depends on catalyst and I cannot add reference to it.
   
   I will try to move SparkSqlParser from core to catalyst and fix this, but will send seperate PR for that



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