[ https://issues.apache.org/jira/browse/SPARK-38870?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Hyukjin Kwon reassigned SPARK-38870: ------------------------------------ Assignee: Furcy Pin > SparkSession.builder returns a new builder in Scala, but not in Python > ---------------------------------------------------------------------- > > Key: SPARK-38870 > URL: https://issues.apache.org/jira/browse/SPARK-38870 > Project: Spark > Issue Type: Bug > Components: PySpark, SQL > Affects Versions: 3.2.1 > Reporter: Furcy Pin > Assignee: Furcy Pin > Priority: Major > > In pyspark, _SparkSession.builder_ always returns the same static builder, > while the expected behaviour should be the same as in Scala, where it returns > a new builder each time. > *How to reproduce* > When we run the following code in Scala : > {code:java} > import org.apache.spark.sql.SparkSession > val s1 = SparkSession.builder.master("local[2]").config("key", > "value").getOrCreate() > println("A : " + s1.conf.get("key")) // value > s1.conf.set("key", "new_value") > println("B : " + s1.conf.get("key")) // new_value > val s2 = SparkSession.builder.getOrCreate() > println("C : " + s1.conf.get("key")) // new_value{code} > The output is : > {code:java} > A : value > B : new_value > C : new_value <<<<<<<<<<<{code} > > But when we run the following (supposedly equivalent) code in Python: > {code:java} > from pyspark.sql import SparkSession > s1 = SparkSession.builder.master("local[2]").config("key", > "value").getOrCreate() > print("A : " + s1.conf.get("key")) > s1.conf.set("key", "new_value") > print("B : " + s1.conf.get("key")) > s2 = SparkSession.builder.getOrCreate() > print("C : " + s1.conf.get("key")){code} > The output is : > {code:java} > A : value > B : new_value > C : value <<<<<<<<<<< > {code} > > > *Root cause analysis* > This comes from the fact that _SparkSession.builder_ behaves differently in > Python than in Scala. In Scala, it returns a *new builder* each time, in > Python it returns *the same builder* every time, and the > SparkSession.Builder._options are static, too. > Because of this, whenever _SparkSession.builder.getOrCreate()_ is called, the > options passed to the very first builder are re-applied every time, and > overrides the option that were set afterwards. > This leads to very awkward behavior in every Spark version up to 3.2.1 > included > {*}Example{*}: > This example crashes, but was fixed by SPARK-37638 > > {code:java} > from pyspark.sql import SparkSession > spark = > SparkSession.builder.config("spark.sql.sources.partitionOverwriteMode", > "DYNAMIC").getOrCreate() > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == > "DYNAMIC" # OK > spark.conf.set("spark.sql.sources.partitionOverwriteMode", "STATIC") > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == "STATIC" > # OK > from pyspark.sql import functions as f > from pyspark.sql.types import StringType > f.col("a").cast(StringType()) > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == "STATIC" > # This fails in all versions until the SPARK-37638 fix > # because before that fix, Column.cast() calle > SparkSession.builder.getOrCreate(){code} > > But this example still crashes in the current version on the master branch > {code:java} > from pyspark.sql import SparkSession > spark = > SparkSession.builder.config("spark.sql.sources.partitionOverwriteMode", > "DYNAMIC").getOrCreate() > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == > "DYNAMIC" # OK > spark.conf.set("spark.sql.sources.partitionOverwriteMode", "STATIC") > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == "STATIC" > # OK > SparkSession.builder.getOrCreate() > assert spark.conf.get("spark.sql.sources.partitionOverwriteMode") == "STATIC" > # This assert fails in master{code} > > I made a Pull Request to fix this bug : > https://github.com/apache/spark/pull/36161 -- This message was sent by Atlassian Jira (v8.20.7#820007) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org