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

[GitHub] [spark] Hisoka-X opened a new pull request, #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Hisoka-X opened a new pull request, #40741:
URL: https://github.com/apache/spark/pull/40741

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   1. add columns field on SQL/SQLCommand in connect protobuf to support send expression to server
   2. parser columns in server to support sql with column
   3. add string_formatter for connect module to support sql with dataframes.
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     4. If you fix some SQL features, you can provide some references of other DBMSes.
     5. If there is design documentation, please add the link.
     6. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   To support  sql with dataframes and columns in spark connect client.
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   Add new test
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   can you try to `cache` and return the dataframe?



##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   can you try to `persist` and return the dataframe?



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


[GitHub] [spark] HyukjinKwon commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   Sorry I completely forgot about this.


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163843165


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Not work, can't pass Analysis stage when use action.



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Hm, can't use just drop by the view by name? IIRC, Spark Connect's `sql` method itself eagerly executes the commands so the views should be created with `clear` is invoked.



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


[GitHub] [spark] HyukjinKwon commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   Oh!
   


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1162863680


##########
connector/connect/common/src/main/protobuf/spark/connect/commands.proto:
##########
@@ -57,6 +57,9 @@ message SqlCommand {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   Add columns field, to support convert column to string on the server, to solve the problem connect client without jvm.



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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1164878200


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -111,6 +111,9 @@ message SQL {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   It can also be merge together, but it doesn't look so clear on the interface.



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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1164155954


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Update: I try to extended `ParameterizedQuery` to add dataframes, but have one blocker is that parser does not recognize `{}` symbols. Extend parser will be a big change for other module. Can I remove support for dataframes in connect client sql temporary? @HyukjinKwon 



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   I think you can work around by calling `df.schema()` that triggers analysis for now, and commenting that this is a workaround to be 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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163757517


##########
python/pyspark/sql/connect/session.py:
##########
@@ -395,13 +397,22 @@ def createDataFrame(
 
     createDataFrame.__doc__ = PySparkSession.createDataFrame.__doc__
 
-    def sql(self, sqlQuery: str, args: Optional[Dict[str, Any]] = None) -> "DataFrame":
-        cmd = SQL(sqlQuery, args)
-        data, properties = self.client.execute_command(cmd.command(self._client))
-        if "sql_command_result" in properties:
-            return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
-        else:
-            return DataFrame.withPlan(SQL(sqlQuery, args), self)
+    def sql(
+        self, sqlQuery: str, args: Optional[Dict[str, Any]] = None, **kwargs: Any
+    ) -> "DataFrame":
+        formatter = SQLStringFormatter(self)
+        if len(kwargs) > 0:
+            sqlQuery = formatter.format(sqlQuery, **kwargs)
+        try:
+            cmd = SQL(sqlQuery, args, formatter.unresolvedColumns)
+            data, properties = self.client.execute_command(cmd.command(self._client))
+            if "sql_command_result" in properties:
+                return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
+            else:
+                return DataFrame.withPlan(SQL(sqlQuery, args, formatter.unresolvedColumns), self)

Review Comment:
   😂 It not work after tried, pyspark maybe worked because it will change plan after called.  But connect will not change plan in dataframe (alway "SELECT" from 'ViewName')



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


[GitHub] [spark] HyukjinKwon commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   cc @ueshin 


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


[GitHub] [spark] Hisoka-X commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   Hi guys, I want implement support sql with columns first, to avoid [sql with dataframes problem](https://github.com/apache/spark/pull/40741#discussion_r1163580352) before have better solution. If ok for you please let me know. Thanks.


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1164870257


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -111,6 +111,9 @@ message SQL {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   There are some different between args and columns. `SELECT {col} FROM t1 WHERE id=1` , the `{col}` used for `columns`. `SELECT id FROM t1 WHERE id= :value `, the `:value` used for `args`. The symbols used by the two are different, and the processing logic is also different. `{col}` is replaced before SQL analysis, and args is value replacement done in SQL analysis.



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


[GitHub] [spark] amaliujia commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -111,6 +111,9 @@ message SQL {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   BTW I am not sure if it can pass proto breaking changes check but re-using proto fields could be a good idea. Maybe originally the type of `args` should already use `Expression`.



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

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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163665874


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Eagerly executes not suit for SELECT query. If the sql are SELECT statement, clear view will cause VIEW_NOT_FOUND



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


[GitHub] [spark] HyukjinKwon commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
python/pyspark/sql/connect/session.py:
##########
@@ -395,13 +397,22 @@ def createDataFrame(
 
     createDataFrame.__doc__ = PySparkSession.createDataFrame.__doc__
 
-    def sql(self, sqlQuery: str, args: Optional[Dict[str, Any]] = None) -> "DataFrame":
-        cmd = SQL(sqlQuery, args)
-        data, properties = self.client.execute_command(cmd.command(self._client))
-        if "sql_command_result" in properties:
-            return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
-        else:
-            return DataFrame.withPlan(SQL(sqlQuery, args), self)
+    def sql(
+        self, sqlQuery: str, args: Optional[Dict[str, Any]] = None, **kwargs: Any
+    ) -> "DataFrame":
+        formatter = SQLStringFormatter(self)
+        if len(kwargs) > 0:
+            sqlQuery = formatter.format(sqlQuery, **kwargs)
+        try:
+            cmd = SQL(sqlQuery, args, formatter.unresolvedColumns)
+            data, properties = self.client.execute_command(cmd.command(self._client))
+            if "sql_command_result" in properties:
+                return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
+            else:
+                return DataFrame.withPlan(SQL(sqlQuery, args, formatter.unresolvedColumns), self)

Review Comment:
   I meant sth like this
   ```python
               if "sql_command_result" in properties:
                   ret = DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
               else:
                   ret = DataFrame.withPlan(SQL(sqlQuery, args, formatter.unresolvedColumns), self)
               if len(kwargs) > 0:
                   ret.schema  # Workaround to drop view blah blah blah by trggering analysis. blah blah
               return ret
   ```



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


[GitHub] [spark] amaliujia commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -111,6 +111,9 @@ message SQL {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   I am wondering if we can merge `map<string, Expression.Literal> args = 2;` and `map<string, Expression.UnresolvedAttribute> columns = 3;` to `map<string, Expression>`?



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

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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163757517


##########
python/pyspark/sql/connect/session.py:
##########
@@ -395,13 +397,22 @@ def createDataFrame(
 
     createDataFrame.__doc__ = PySparkSession.createDataFrame.__doc__
 
-    def sql(self, sqlQuery: str, args: Optional[Dict[str, Any]] = None) -> "DataFrame":
-        cmd = SQL(sqlQuery, args)
-        data, properties = self.client.execute_command(cmd.command(self._client))
-        if "sql_command_result" in properties:
-            return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
-        else:
-            return DataFrame.withPlan(SQL(sqlQuery, args), self)
+    def sql(
+        self, sqlQuery: str, args: Optional[Dict[str, Any]] = None, **kwargs: Any
+    ) -> "DataFrame":
+        formatter = SQLStringFormatter(self)
+        if len(kwargs) > 0:
+            sqlQuery = formatter.format(sqlQuery, **kwargs)
+        try:
+            cmd = SQL(sqlQuery, args, formatter.unresolvedColumns)
+            data, properties = self.client.execute_command(cmd.command(self._client))
+            if "sql_command_result" in properties:
+                return DataFrame.withPlan(CachedRelation(properties["sql_command_result"]), self)
+            else:
+                return DataFrame.withPlan(SQL(sqlQuery, args, formatter.unresolvedColumns), self)

Review Comment:
   😂 It not work after tried, pyspark maybe worked because it will change plan after called.  But connect will not change plan in dataframe



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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163580352


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   There is something wrong with this version of the code
   1. There is no way to drop view normally. Because the view is to pass the name into the sql for reference, not the logical plan of the reference, if the view is dropped before the action, the view cannot be processed normally. The difference between this and the original pyspark program is that the original pyspark saves the parsed logical plan, but now we save the name of the view. Connect has no way to save the parsed logical plan. If it does so, it will increase the complexity of the client side, because the original logical plan and the protobuf type plan must be maintained at the same time.
   2. In order to solve this problem, `ParameterizedQuery` needs to be extended to support adding logical plans as parameters, just like the current `args`
   
   @HyukjinKwon @cloud-fan @MaxGekk WDYT?



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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163691632


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Sorry, I don't get the point. Where to call `df.schema()`? And the calling will resolved what problem? Like this?
   <img width="401" alt="image" src="https://user-images.githubusercontent.com/32387433/231373441-75bb89a8-2c90-4ad0-9a4b-0cf60421dd86.png">
   



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


[GitHub] [spark] Hisoka-X commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   @HyukjinKwon Never mind 😀. For me as long as the problem is solved, all is fine.


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


[GitHub] [spark] Hisoka-X closed pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X closed pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns
URL: https://github.com/apache/spark/pull/40741


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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1163665874


##########
python/pyspark/sql/connect/sql_formatter.py:
##########
@@ -0,0 +1,82 @@
+#
+# 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.
+#
+
+import string
+import typing
+from typing import Any, Optional, List, Tuple, Sequence, Mapping
+import uuid
+
+from pyspark.sql.connect.proto import Expression
+
+if typing.TYPE_CHECKING:
+    from pyspark.sql.connect import SparkSession, DataFrame
+from pyspark.sql.connect.functions import lit
+
+
+class SQLStringFormatter(string.Formatter):
+    """
+    A standard ``string.Formatter`` in Python that can understand PySpark instances
+    with basic Python objects. This object has to be clear after the use for single SQL
+    query; cannot be reused across multiple SQL queries without cleaning.
+    """
+
+    def __init__(self, session: "SparkSession") -> None:
+        self._session: "SparkSession" = session
+        self._temp_views: List[Tuple[DataFrame, str]] = []
+        self._unresolvedColumns: typing.Dict[str, Expression] = {}
+
+    def get_field(self, field_name: str, args: Sequence[Any], kwargs: Mapping[str, Any]) -> Any:
+        obj, first = super(SQLStringFormatter, self).get_field(field_name, args, kwargs)
+        return self._convert_value(obj, field_name), first
+
+    def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
+        """
+        Converts the given value into a SQL string.
+        """
+        from pyspark.sql.connect import Column, DataFrame
+
+        if isinstance(val, DataFrame):
+            for df, n in self._temp_views:
+                if df is val:
+                    return n
+            df_name = "_pyspark_%s" % str(uuid.uuid4()).replace("-", "")
+            self._temp_views.append((val, df_name))
+            val.createOrReplaceTempView(df_name)
+            return df_name
+        if isinstance(val, Column):
+            for f, _ in self._unresolvedColumns.items():
+                if f is field_name:
+                    break
+            self._unresolvedColumns[field_name] = val.expr.to_plan(self._session.client)
+            return wrap(field_name)
+        elif isinstance(val, str):
+            self._unresolvedColumns[field_name] = lit(val).expr.to_plan(self._session.client)
+            return wrap(field_name)
+        return val
+
+    def clear(self) -> None:
+        """TODO We need to find a better time to drop view"""

Review Comment:
   Eagerly executes not suit for SELECT query. If the sql are SELECT statement, drop view will cause VIEW_NOT_FOUND



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


[GitHub] [spark] amaliujia commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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


##########
connector/connect/common/src/main/protobuf/spark/connect/relations.proto:
##########
@@ -111,6 +111,9 @@ message SQL {
 
   // (Optional) A map of parameter names to literal expressions.
   map<string, Expression.Literal> args = 2;
+
+  // (Optional) A map of parameter names to column expressions.
+  map<string, Expression.UnresolvedAttribute> columns = 3;

Review Comment:
   I am wondering if we can merge `map<string, Expression.Literal> args = 2;` and `map<string, Expression.UnresolvedAttribute> columns = 3;` to `map<string, Expressions> args = 2`?



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


[GitHub] [spark] Hisoka-X commented on a diff in pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

Posted by "Hisoka-X (via GitHub)" <gi...@apache.org>.
Hisoka-X commented on code in PR #40741:
URL: https://github.com/apache/spark/pull/40741#discussion_r1162860013


##########
python/pyspark/sql/tests/connect/test_connect_basic.py:
##########
@@ -2039,6 +2039,31 @@ def test_repr(self):
             self.spark.sql(query).__repr__(),
         )
 
+    def test_string_sql_formatter(self):
+        self.connect.sql(
+            "SELECT * FROM range(10) WHERE id > {bound1} AND id < {bound2}", bound1=7, bound2=9
+        ).show()
+
+        mydf = self.connect.range(10)
+        self.connect.sql(
+            "SELECT {col} FROM {mydf} WHERE id IN {x}", col=mydf.id, mydf=mydf, x=tuple(range(4))
+        ).show()
+
+        self.connect.sql(

Review Comment:
   The test with dataframes can't passed on my env, I trying figure it out.



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


[GitHub] [spark] HyukjinKwon commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   cc @MaxGekk 


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


[GitHub] [spark] HyukjinKwon commented on pull request #40741: [SPARK-41811][CONNECT][CLIENT] Support sql with dataframes and columns

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

   I totally forgot that we have an ongoing pr


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