Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-43389][SQL] Added a null check for lineSep option #41904

Closed
wants to merge 4 commits into from

Conversation

gdhuper
Copy link
Contributor

@gdhuper gdhuper commented Jul 9, 2023

What changes were proposed in this pull request?

Why are the changes needed?

Does this PR introduce any user-facing change?

Users now should be able to explicitly set lineSep as None without getting an exception
After some discussion, it was decided to add a require check for null instead of letting it through.

How was this patch tested?

Tested the changes with a python script that explicitly sets lineSep to None

from pyspark.sql import SparkSession

# Create a SparkSession
spark = SparkSession.builder.appName("HelloWorld").getOrCreate()

# Read CSV into a DataFrame
df = spark.read.csv("/tmp/hello.csv", header=True, inferSchema=True, lineSep=None)

# Also tested the following case when options are passed before invoking .csv
#df = spark.read.option("lineSep", None).csv("/Users/gdhuper/Documents/tmp/hello.csv", header=True, inferSchema=True)

# Show the DataFrame
df.show()

# Stop the SparkSession
spark.stop()

@github-actions github-actions bot added the SQL label Jul 9, 2023
@gdhuper gdhuper marked this pull request as ready for review July 9, 2023 09:22
"with 2 characters due to the limitation of supporting multi-char 'lineSep' within quotes.")
sep
val lineSeparator: Option[String] = parameters.get(LINE_SEP) match {
case Some(sep) if sep != null =>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is a null line separator even valid? I'd imagine that should be an error, if an empty one is

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As far as my understanding, None when passed through python (lineSep: None) ends up being null in this function. Hence, I have a check for null.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

None isn't the default, right? if a user passes None, it feels like that should be an error

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@srowen I can see that it should be an error just like an empty value ''.
However, one of the use cases where this might be helpful is where someone might need to dynamically switch the value for lineSep from None to something else. For example, code generation is one example that comes to mind, where a base template with lineSep: None is used that can be replaced with appropriate values as desired.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hm, but generated code would not need to set 'None' right? that isn't meaningful at the point you actually use Spark

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see your point. I guess we could also use another placeholder value instead of None in that scenario.
I can revert the changes and add a require clause for null check instead of letting it through. Or would you rather have it throw an NullPointerException?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd just require() it for consistency, unless there's an argument for other handling. (If there is, it'd probably apply to the "" case too)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done.

@HyukjinKwon HyukjinKwon changed the title [SPARK-43389] [PySpark, SQL] Added a null check for lineSep option [SPARK-43389][PYTHON][SQL] Added a null check for lineSep option Jul 10, 2023
@HyukjinKwon HyukjinKwon changed the title [SPARK-43389][PYTHON][SQL] Added a null check for lineSep option [SPARK-43389][SQL] Added a null check for lineSep option Jul 10, 2023
@HyukjinKwon
Copy link
Member

and I actually think we might need to check every option. In some options, None might make sense as a valid option but others do not.

@srowen
Copy link
Member

srowen commented Jul 13, 2023

Merged to master

@srowen srowen closed this in 9f07e4a Jul 13, 2023
@srowen
Copy link
Member

srowen commented Jul 13, 2023

@gdhuper what's your JIRA handle? I can assign it to you

@gdhuper
Copy link
Contributor Author

gdhuper commented Jul 13, 2023

@gdhuper what's your JIRA handle? I can assign it to you

gdhuper

ragnarok56 pushed a commit to ragnarok56/spark that referenced this pull request Mar 2, 2024
### What changes were proposed in this pull request?

### Why are the changes needed?

- `spark.read.csv` throws `NullPointerException` when lineSep is set to None
- More details about the issue here: https://issues.apache.org/jira/browse/SPARK-43389

### Does this PR introduce _any_ user-facing change?

~~Users now should be able to explicitly set `lineSep` as `None` without getting an exception~~
After some discussion, it was decided to add a `require` check for `null` instead of letting it through.

### How was this patch tested?

Tested the changes with a python script that explicitly sets `lineSep` to `None`
```python
from pyspark.sql import SparkSession

# Create a SparkSession
spark = SparkSession.builder.appName("HelloWorld").getOrCreate()

# Read CSV into a DataFrame
df = spark.read.csv("/tmp/hello.csv", header=True, inferSchema=True, lineSep=None)

# Also tested the following case when options are passed before invoking .csv
#df = spark.read.option("lineSep", None).csv("/Users/gdhuper/Documents/tmp/hello.csv", header=True, inferSchema=True)

# Show the DataFrame
df.show()

# Stop the SparkSession
spark.stop()
```

Closes apache#41904 from gdhuper/gdhuper/SPARK-43389.

Authored-by: Gurpreet Singh <gdhuper@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
3 participants