[ 
https://issues.apache.org/jira/browse/SPARK-50126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17922879#comment-17922879
 ] 

Sakthi commented on SPARK-50126:
--------------------------------

Hello all, I would like to get this issue fixed in the branch-3.5.
 * Looks like the SQLExpression class is missing the __repr__ method for the 
string representation.
 * In the master branch this issue seems to have been fixed via this 
[JIRA|https://issues.apache.org/jira/browse/SPARK-48276] and this 
[commit|https://github.com/apache/spark/commit/d31161b27404219169345c716d7b7fe20356085d]

I tried cherry-picking the commit and verified the above mentioned error 
symptoms as shown below:



PySpark with Spark Connect
{code:java}
❯ pyspark --remote "sc://localhost:15002"
Python 3.11.7 (main, Jan 24 2024, 21:22:12) [GCC 7.3.1 20180712 (Red Hat 
7.3.1-17)] on linux
Type "help", "copyright", "credits" or "license" for more information.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 3.5.5.dev0
      /_/Using Python version 3.11.7 (main, Jan 24 2024 21:22:12)
Client connected to the Spark Connect server at localhost:15002
SparkSession available as 'spark'.
>>> from pyspark.sql.functions import expr
>>> e = expr("*")
>>> print(e)
Column<'*'> {code}
 

PySpark without Spark Connect

 
{code:java}
❯ pyspark
Python 3.11.7 (main, Jan 24 2024, 21:22:12) [GCC 7.3.1 20180712 (Red Hat 
7.3.1-17)] on linux
Type "help", "copyright", "credits" or "license" for more information.
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use 
setLogLevel(newLevel).
25/01/31 23:06:15 WARN NativeCodeLoader: Unable to load native-hadoop library 
for your platform... using builtin-java classes where applicable
25/01/31 23:06:16 WARN Utils: Service 'SparkUI' could not bind on port 4040. 
Attempting port 4041.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 3.5.5-SNAPSHOT
      /_/Using Python version 3.11.7 (main, Jan 24 2024 21:22:12)
Spark context Web UI available at http://...:4041
Spark context available as 'sc' (master = local[*], app id = 
local-1738364776068).
SparkSession available as 'spark'.
>>> from pyspark.sql.functions import expr
>>> e = expr("*")
>>> print(e)
Column<'unresolvedstar()'>{code}
 

 

I just opened a PR ([https://github.com/apache/spark/pull/49755)] with the 
above fix. Let me know if things look okay.

> PySpark expr() (expression) SQL Function returns None in Spark Connect
> ----------------------------------------------------------------------
>
>                 Key: SPARK-50126
>                 URL: https://issues.apache.org/jira/browse/SPARK-50126
>             Project: Spark
>          Issue Type: Bug
>          Components: Connect, PySpark
>    Affects Versions: 3.5.0, 3.5.1, 3.5.2, 3.5.3
>         Environment: Spark 3.5.3
> Python 3.9.20
>            Reporter: Nick Herzig
>            Priority: Major
>              Labels: connect, expression, pull-request-available, pyspark, 
> pyspark.sql.functions, remote
>
> When PySpark is launched with remote Spark Connect configuration, calls to 
> pyspark.sql.functions.expr always return `Column<None>`.
>  
> PySpark without Spark Connect
> {code:java}
> $ pyspark
> ...
> >>> from pyspark.sql.functions import expr
> >>> e = expr("*")
> >>> print(e)
> Column<'unresolvedstar()'>
> {code}
> PySpark with Spark Connect
> {code:java}
> $ pyspark --remote "sc://localhost:15002"
> ...
> Client connected to the Spark Connect server at localhost:15002
> SparkSession available as 'spark'.
> >>> from pyspark.sql.functions import expr
> >>> e = expr("*")
> >>> print(e)
> Column<'None'>
> {code}
> Additional example
> {code:java}
> $ pyspark
> ...
> Spark context available as 'sc' (master = local[*], app id = 
> local-1729876117745).
> SparkSession available as 'spark'.
> >>> from pyspark.sql.functions import expr
> >>> expr("*")
> Column<'unresolvedstar()'>
> >>> SparkSession.builder.master("local[*]").getOrCreate().stop()
> >>> spark = SparkSession.builder.remote("sc://localhost:15002").getOrCreate()
> >>> expr("*")
> Column<'None'> {code}
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

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

Reply via email to