Skip to content

Commit

Permalink
[SPARK-46232][PYTHON] Migrate all remaining ValueError into PySpark e…
Browse files Browse the repository at this point in the history
…rror framework

### What changes were proposed in this pull request?

This PR proposes to migrate all remaining `ValueError`  from `pyspark/sql/*` into PySpark error framework, `PySparkValueError` with assigning dedicated error classes.

### Why are the changes needed?

To improve the error handling in PySpark.

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

No API changes, but the user-facing error messages will be improved.

### How was this patch tested?

The existing CI should pass.

### Was this patch authored or co-authored using generative AI tooling?

No.

Closes #44149 from itholic/migrate_value_error.

Authored-by: Haejoon Lee <haejoon.lee@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
  • Loading branch information
itholic authored and dongjoon-hyun committed Dec 4, 2023
1 parent 6f112f7 commit b23ae15
Show file tree
Hide file tree
Showing 5 changed files with 38 additions and 12 deletions.
19 changes: 17 additions & 2 deletions python/pyspark/errors/error_classes.py
Original file line number Diff line number Diff line change
Expand Up @@ -287,6 +287,11 @@
"NumPy array input should be of <dimensions> dimensions."
]
},
"INVALID_NUMBER_OF_DATAFRAMES_IN_GROUP" : {
"message" : [
"Invalid number of dataframes in group <dataframes_in_group>."
]
},
"INVALID_PANDAS_UDF" : {
"message" : [
"Invalid function: <detail>"
Expand Down Expand Up @@ -803,9 +808,9 @@
"Expected <expected> values for `<item>`, got <actual>."
]
},
"TYPE_HINT_REQUIRED" : {
"TYPE_HINT_SHOULD_BE_SPECIFIED" : {
"message" : [
"A <arg_type> is required <where>."
"Type hints for <target> should be specified; however, got <sig>."
]
},
"UDF_RETURN_TYPE" : {
Expand Down Expand Up @@ -888,6 +893,11 @@
"Unknown response: <response>."
]
},
"UNKNOWN_VALUE_FOR" : {
"message" : [
"Unknown value for `<var>`."
]
},
"UNSUPPORTED_DATA_TYPE" : {
"message" : [
"Unsupported DataType `<data_type>`."
Expand Down Expand Up @@ -983,6 +993,11 @@
"Value for `<arg_name>` only supports the 'pearson', got '<arg_value>'."
]
},
"VALUE_NOT_PLAIN_COLUMN_REFERENCE" : {
"message" : [
"Value <val> in <field_name> should be a plain column reference such as `df.col` or `col('column')`."
]
},
"VALUE_NOT_POSITIVE" : {
"message" : [
"Value for `<arg_name>` must be positive, got '<arg_value>'."
Expand Down
5 changes: 3 additions & 2 deletions python/pyspark/sql/pandas/serializers.py
Original file line number Diff line number Diff line change
Expand Up @@ -707,8 +707,9 @@ def load_stream(self, stream):
yield batches1, batches2

elif dataframes_in_group != 0:
raise ValueError(
"Invalid number of dataframes in group {0}".format(dataframes_in_group)
raise PySparkValueError(
error_class="INVALID_NUMBER_OF_DATAFRAMES_IN_GROUP",
message_parameters={"dataframes_in_group": str(dataframes_in_group)},
)


Expand Down
12 changes: 9 additions & 3 deletions python/pyspark/sql/pandas/typehints.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
from typing import Any, Callable, Dict, Optional, Union, TYPE_CHECKING

from pyspark.sql.pandas.utils import require_minimum_pandas_version
from pyspark.errors import PySparkNotImplementedError
from pyspark.errors import PySparkNotImplementedError, PySparkValueError

if TYPE_CHECKING:
from pyspark.sql.pandas._typing import (
Expand Down Expand Up @@ -51,12 +51,18 @@ def infer_eval_type(
annotations[parameter] for parameter in sig.parameters if parameter in annotations
]
if len(parameters_sig) != len(sig.parameters):
raise ValueError("Type hints for all parameters should be specified; however, got %s" % sig)
raise PySparkValueError(
error_class="TYPE_HINT_SHOULD_BE_SPECIFIED",
message_parameters={"target": "all parameters", "sig": str(sig)},
)

# Check if the return has a type hint
return_annotation = type_hints.get("return", sig.return_annotation)
if sig.empty is return_annotation:
raise ValueError("Type hint for the return type should be specified; however, got %s" % sig)
raise PySparkValueError(
error_class="TYPE_HINT_SHOULD_BE_SPECIFIED",
message_parameters={"target": "the return type", "sig": str(sig)},
)

# Series, Frame or Union[DataFrame, Series], ... -> Series or Frame
is_series_or_frame = all(
Expand Down
7 changes: 5 additions & 2 deletions python/pyspark/sql/pandas/types.py
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
UserDefinedType,
_create_row,
)
from pyspark.errors import PySparkTypeError, UnsupportedOperationException
from pyspark.errors import PySparkTypeError, UnsupportedOperationException, PySparkValueError

if TYPE_CHECKING:
import pandas as pd
Expand Down Expand Up @@ -716,7 +716,10 @@ def convert_struct_as_dict(value: Any) -> Any:
return convert_struct_as_dict

else:
raise ValueError(f"Unknown value for `struct_in_pandas`: {_struct_in_pandas}")
raise PySparkValueError(
error_class="UNKNOWN_VALUE_FOR",
message_parameters={"var": str(_struct_in_pandas)},
)

elif isinstance(dt, TimestampType):
assert timezone is not None
Expand Down
7 changes: 4 additions & 3 deletions python/pyspark/sql/sql_formatter.py
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
if typing.TYPE_CHECKING:
from pyspark.sql import SparkSession, DataFrame
from pyspark.sql.functions import lit
from pyspark.errors import PySparkValueError


class SQLStringFormatter(string.Formatter):
Expand Down Expand Up @@ -61,9 +62,9 @@ def _convert_value(self, val: Any, field_name: str) -> Optional[str]:
):
return jexpr.sql()
else:
raise ValueError(
"%s in %s should be a plain column reference such as `df.col` "
"or `col('column')`" % (val, field_name)
raise PySparkValueError(
error_class="VALUE_NOT_PLAIN_COLUMN_REFERENCE",
message_parameters={"val": str(val), "field_name": field_name},
)
elif isinstance(val, DataFrame):
for df, n in self._temp_views:
Expand Down

0 comments on commit b23ae15

Please sign in to comment.