diff --git a/python/pyspark/errors/error_classes.py b/python/pyspark/errors/error_classes.py index ab65600eb31ce..9d0f1c6e769a2 100644 --- a/python/pyspark/errors/error_classes.py +++ b/python/pyspark/errors/error_classes.py @@ -39,99 +39,99 @@ "Function `` should return Column, got ." ] }, - "NOT_AN_INTEGER" : { + "NOT_BOOL" : { "message" : [ - "Argument `` should be a integer, got ." + "Argument `` should be a bool, got ." ] }, - "NOT_A_BOOLEAN" : { + "NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_LIST_OR_STR_OR_TUPLE" : { "message" : [ - "Argument `` should be a boolean, got ." + "Argument `` should be a bool, dict, float, int, str or tuple, got ." ] }, - "NOT_A_COLUMN" : { + "NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_STR" : { "message" : [ - "Argument `` should be a Column, got ." + "Argument `` should be a bool, dict, float, int or str, got ." ] }, - "NOT_A_DATAFRAME" : { + "NOT_BOOL_OR_LIST" : { "message" : [ - "Argument `` must be a DataFrame, got ." + "Argument `` should be a bool or list, got ." ] }, - "NOT_A_DICT" : { + "NOT_BOOL_OR_STR" : { "message" : [ - "Argument `` should be a dict, got ." + "Argument `` should be a bool or str, got ." ] }, - "NOT_A_STRING" : { + "NOT_COLUMN" : { "message" : [ - "Argument `` should be a str, got ." + "Argument `` should be a Column, got ." ] }, - "NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING_OR_TUPLE" : { + "NOT_COLUMN_OR_DATATYPE_OR_STR" : { "message" : [ - "Argument `` should be a boolean, dict, float, integer, string or tuple, got ." + "Argument `` should be a Column, str or DataType, but got ." ] }, - "NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_STRING" : { + "NOT_COLUMN_OR_FLOAT_OR_INT_OR_LIST_OR_STR" : { "message" : [ - "Argument `` should be a boolean, dict, float, integer or string, got ." + "Argument `` should be a column, float, integer, list or string, got ." ] }, - "NOT_BOOL_OR_LIST" : { + "NOT_COLUMN_OR_INT" : { "message" : [ - "Argument `` should be a boolean or list, got ." + "Argument `` should be a Column or int, got ." ] }, - "NOT_BOOL_OR_STRING" : { + "NOT_COLUMN_OR_INT_OR_STR" : { "message" : [ - "Argument `` should be a boolean or string, got ." + "Argument `` should be a Column, int or str, got ." ] }, - "NOT_COLUMN_OR_DATATYPE_OR_STRING" : { + "NOT_COLUMN_OR_STR" : { "message" : [ - "Argument `` should be a Column or str or DataType, but got ." + "Argument `` should be a Column or str, got ." ] }, - "NOT_COLUMN_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING" : { + "NOT_DATAFRAME" : { "message" : [ - "Argument `` should be a column, float, integer, list or string, got ." + "Argument `` must be a DataFrame, got ." ] }, - "NOT_COLUMN_OR_INTEGER" : { + "NOT_DICT" : { "message" : [ - "Argument `` should be a Column or int, got ." + "Argument `` should be a dict, got ." ] }, - "NOT_COLUMN_OR_INTEGER_OR_STRING" : { + "NOT_FLOAT_OR_INT" : { "message" : [ - "Argument `` should be a Column, int or str, got ." + "Argument `` should be a float or int, got ." ] }, - "NOT_COLUMN_OR_STRING" : { + "NOT_FLOAT_OR_INT_OR_LIST_OR_STR" : { "message" : [ - "Argument `` should be a Column or str, got ." + "Argument `` should be a float, int, list or str, got ." ] }, - "NOT_FLOAT_OR_INTEGER" : { + "NOT_INT" : { "message" : [ - "Argument `` should be a float or integer, got ." + "Argument `` should be an int, got ." ] }, - "NOT_FLOAT_OR_INTEGER_OR_LIST_OR_STRING" : { + "NOT_LIST_OR_STR_OR_TUPLE" : { "message" : [ - "Argument `` should be a float, integer, list or string, got ." + "Argument `` should be a list, str or tuple, got ." ] }, - "NOT_LIST_OR_STRING_OR_TUPLE" : { + "NOT_LIST_OR_TUPLE" : { "message" : [ - "Argument `` should be a list, string or tuple, got ." + "Argument `` should be a list or tuple, got ." ] }, - "NOT_LIST_OR_TUPLE" : { + "NOT_STR" : { "message" : [ - "Argument `` should be a list or tuple, got ." + "Argument `` should be a str, got ." ] }, "UNSUPPORTED_NUMPY_ARRAY_SCALAR" : { @@ -141,17 +141,17 @@ }, "UNSUPPORTED_PARAM_TYPE_FOR_HIGHER_ORDER_FUNCTION" : { "message" : [ - "Function `` should use only POSITIONAL or POSITIONAL OR KEYWORD arguments" + "Function `` should use only POSITIONAL or POSITIONAL OR KEYWORD arguments." ] }, "WRONG_NUM_ARGS_FOR_HIGHER_ORDER_FUNCTION" : { "message" : [ - "Function `` should take between 1 and 3 arguments, but provided function takes " + "Function `` should take between 1 and 3 arguments, but provided function takes ." ] }, "WRONG_NUM_COLUMNS" : { "message" : [ - "Function `` should take at least columns" + "Function `` should take at least columns." ] } } diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 482c48f0c915e..42ea448e5d69d 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -143,7 +143,7 @@ def alias(self, alias: str) -> "DataFrame": def colRegex(self, colName: str) -> Column: if not isinstance(colName, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "colName", "arg_type": type(colName).__name__}, ) return Column(UnresolvedRegex(colName)) @@ -232,7 +232,7 @@ def repartition( # type: ignore[misc] ) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={ "arg_name": "numPartitions", "arg_type": type(numPartitions).__name__, @@ -574,7 +574,7 @@ def withColumnRenamed(self, existing: str, new: str) -> "DataFrame": def withColumnsRenamed(self, colsMap: Dict[str, str]) -> "DataFrame": if not isinstance(colsMap, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "colsMap", "arg_type": type(colsMap).__name__}, ) @@ -587,12 +587,12 @@ def _show_string( ) -> str: if not isinstance(n, int) or isinstance(n, bool): raise PySparkTypeError( - error_class="NOT_AN_INTEGER", + error_class="NOT_INT", message_parameters={"arg_name": "n", "arg_type": type(n).__name__}, ) if not isinstance(vertical, bool): raise PySparkTypeError( - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={"arg_name": "vertical", "arg_type": type(vertical).__name__}, ) @@ -604,7 +604,7 @@ def _show_string( _truncate = int(truncate) except ValueError: raise PySparkTypeError( - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={ "arg_name": "truncate", "arg_type": type(truncate).__name__, @@ -836,7 +836,7 @@ def intersectAll(self, other: "DataFrame") -> "DataFrame": def where(self, condition: Union[Column, str]) -> "DataFrame": if not isinstance(condition, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "condition", "arg_type": type(condition).__name__}, ) return self.filter(condition) @@ -856,7 +856,7 @@ def fillna( ) -> "DataFrame": if not isinstance(value, (float, int, str, bool, dict)): raise PySparkTypeError( - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_STRING", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_STR", message_parameters={"arg_name": "value", "arg_type": type(value).__name__}, ) if isinstance(value, dict): @@ -943,7 +943,7 @@ def dropna( _cols = list(subset) else: raise PySparkTypeError( - error_class="NOT_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "subset", "arg_type": type(subset).__name__}, ) @@ -997,7 +997,7 @@ def all_of_(xs: Iterable) -> bool: valid_types = (bool, float, int, str, list, tuple) if not isinstance(to_replace, valid_types + (dict,)): raise PySparkTypeError( - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_LIST_OR_STR_OR_TUPLE", message_parameters={ "arg_name": "to_replace", "arg_type": type(to_replace).__name__, @@ -1097,12 +1097,12 @@ def describe(self, *cols: Union[str, List[str]]) -> "DataFrame": def cov(self, col1: str, col2: str) -> float: if not isinstance(col1, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col1", "arg_type": type(col1).__name__}, ) if not isinstance(col2, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col2", "arg_type": type(col2).__name__}, ) pdf = DataFrame.withPlan( @@ -1224,12 +1224,12 @@ def sampleBy( col = Column(ColumnReference(col)) elif not isinstance(col, Column): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "col", "arg_type": type(col).__name__}, ) if not isinstance(fractions, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "fractions", "arg_type": type(fractions).__name__}, ) for k, v in fractions.items(): diff --git a/python/pyspark/sql/connect/functions.py b/python/pyspark/sql/connect/functions.py index 3b3036f3be8e3..96ef9aaf7ff20 100644 --- a/python/pyspark/sql/connect/functions.py +++ b/python/pyspark/sql/connect/functions.py @@ -262,7 +262,7 @@ def broadcast(df: "DataFrame") -> "DataFrame": if not isinstance(df, DataFrame): raise PySparkTypeError( - error_class="NOT_A_DATAFRAME", + error_class="NOT_DATAFRAME", message_parameters={"arg_name": "df", "arg_type": type(df).__name__}, ) return df.hint("broadcast") @@ -375,7 +375,7 @@ def when(condition: Column, value: Any) -> Column: # Explicitly not using ColumnOrName type here to make reading condition less opaque if not isinstance(condition, Column): raise PySparkTypeError( - error_class="NOT_A_COLUMN", + error_class="NOT_COLUMN", message_parameters={"arg_name": "condition", "arg_type": type(condition).__name__}, ) @@ -1357,7 +1357,7 @@ def from_csv( _schema = lit(schema) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "schema", "arg_type": type(schema).__name__}, ) @@ -1383,7 +1383,7 @@ def from_json( _schema = lit(schema) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_DATATYPE_OR_STRING", + error_class="NOT_COLUMN_OR_DATATYPE_OR_STR", message_parameters={"arg_name": "schema", "arg_type": type(schema).__name__}, ) @@ -1544,7 +1544,7 @@ def schema_of_csv(csv: "ColumnOrName", options: Optional[Dict[str, str]] = None) _csv = lit(csv) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "csv", "arg_type": type(csv).__name__}, ) @@ -1564,7 +1564,7 @@ def schema_of_json(json: "ColumnOrName", options: Optional[Dict[str, str]] = Non _json = lit(json) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "json", "arg_type": type(json).__name__}, ) @@ -1600,7 +1600,7 @@ def slice( _start = lit(start) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "start", "arg_type": type(start).__name__}, ) @@ -1610,7 +1610,7 @@ def slice( _length = lit(length) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "length", "arg_type": type(length).__name__}, ) @@ -1802,12 +1802,12 @@ def overlay( ) -> Column: if not isinstance(pos, (int, str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "pos", "arg_type": type(pos).__name__}, ) if len is not None and not isinstance(len, (int, str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "len", "arg_type": type(len).__name__}, ) @@ -2224,7 +2224,7 @@ def window( ) -> Column: if windowDuration is None or not isinstance(windowDuration, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={ "arg_name": "windowDuration", "arg_type": type(windowDuration).__name__, @@ -2232,7 +2232,7 @@ def window( ) if slideDuration is not None and not isinstance(slideDuration, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={ "arg_name": "slideDuration", "arg_type": type(slideDuration).__name__, @@ -2240,7 +2240,7 @@ def window( ) if startTime is not None and not isinstance(startTime, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "startTime", "arg_type": type(startTime).__name__}, ) @@ -2275,7 +2275,7 @@ def window_time( def session_window(timeColumn: "ColumnOrName", gapDuration: Union[Column, str]) -> Column: if gapDuration is None or not isinstance(gapDuration, (Column, str)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "gapDuration", "arg_type": type(gapDuration).__name__}, ) @@ -2300,7 +2300,7 @@ def bucket(numBuckets: Union[Column, int], col: "ColumnOrName") -> Column: _numBuckets = numBuckets else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER", + error_class="NOT_COLUMN_OR_INT", message_parameters={ "arg_name": "numBuckets", "arg_type": type(numBuckets).__name__, @@ -2348,7 +2348,7 @@ def assert_true(col: "ColumnOrName", errMsg: Optional[Union[Column, str]] = None return _invoke_function_over_columns("assert_true", col) if not isinstance(errMsg, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": type(errMsg).__name__}, ) _err_msg = lit(errMsg) if isinstance(errMsg, str) else _to_col(errMsg) @@ -2361,7 +2361,7 @@ def assert_true(col: "ColumnOrName", errMsg: Optional[Union[Column, str]] = None def raise_error(errMsg: Union[Column, str]) -> Column: if not isinstance(errMsg, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": type(errMsg).__name__}, ) _err_msg = lit(errMsg) if isinstance(errMsg, str) else _to_col(errMsg) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f9d9c199fafbe..2e15cb6149f84 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -695,7 +695,7 @@ def explain( if not (is_no_argument or is_extended_case or is_extended_as_mode or is_mode_case): if (extended is not None) and (not isinstance(extended, (bool, str))): raise PySparkTypeError( - error_class="NOT_BOOL_OR_STRING", + error_class="NOT_BOOL_OR_STR", message_parameters={ "arg_name": "extended", "arg_type": type(extended).__name__, @@ -703,7 +703,7 @@ def explain( ) if (mode is not None) and (not isinstance(mode, str)): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "mode", "arg_type": type(mode).__name__}, ) @@ -901,13 +901,13 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = if not isinstance(n, int) or isinstance(n, bool): raise PySparkTypeError( - error_class="NOT_AN_INTEGER", + error_class="NOT_INT", message_parameters={"arg_name": "n", "arg_type": type(n).__name__}, ) if not isinstance(vertical, bool): raise PySparkTypeError( - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={"arg_name": "vertical", "arg_type": type(vertical).__name__}, ) @@ -918,7 +918,7 @@ def show(self, n: int = 20, truncate: Union[bool, int] = True, vertical: bool = int_truncate = int(truncate) except ValueError: raise PySparkTypeError( - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={ "arg_name": "truncate", "arg_type": type(truncate).__name__, @@ -1104,12 +1104,12 @@ def withWatermark(self, eventTime: str, delayThreshold: str) -> "DataFrame": """ if not eventTime or type(eventTime) is not str: raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "eventTime", "arg_type": type(eventTime).__name__}, ) if not delayThreshold or type(delayThreshold) is not str: raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={ "arg_name": "delayThreshold", "arg_type": type(delayThreshold).__name__, @@ -1163,7 +1163,7 @@ def hint( if not isinstance(name, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "name", "arg_type": type(name).__name__}, ) @@ -1657,7 +1657,7 @@ def repartition( # type: ignore[misc] return DataFrame(self._jdf.repartition(self._jcols(*cols)), self.sparkSession) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={ "arg_name": "numPartitions", "arg_type": type(numPartitions).__name__, @@ -1730,7 +1730,7 @@ def repartitionByRange( # type: ignore[misc] return DataFrame(self._jdf.repartitionByRange(self._jcols(*cols)), self.sparkSession) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={ "arg_name": "numPartitions", "arg_type": type(numPartitions).__name__, @@ -1910,12 +1910,12 @@ def sampleBy( col = Column(col) elif not isinstance(col, Column): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "col", "arg_type": type(col).__name__}, ) if not isinstance(fractions, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "fractions", "arg_type": type(fractions).__name__}, ) for k, v in fractions.items(): @@ -2061,7 +2061,7 @@ def colRegex(self, colName: str) -> Column: """ if not isinstance(colName, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "colName", "arg_type": type(colName).__name__}, ) jc = self._jdf.colRegex(colName) @@ -2890,7 +2890,7 @@ def __getitem__(self, item: Union[int, str, Column, List, Tuple]) -> Union[Colum return Column(jc) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING", + error_class="NOT_COLUMN_OR_FLOAT_OR_INT_OR_LIST_OR_STR", message_parameters={"arg_name": "item", "arg_type": type(item).__name__}, ) @@ -3089,7 +3089,7 @@ def filter(self, condition: "ColumnOrName") -> "DataFrame": jdf = self._jdf.filter(condition._jc) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "condition", "arg_type": type(condition).__name__}, ) return DataFrame(jdf, self.sparkSession) @@ -3937,7 +3937,7 @@ def dropna( subset = [subset] elif not isinstance(subset, (list, tuple)): raise PySparkTypeError( - error_class="NOT_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "subset", "arg_type": type(subset).__name__}, ) @@ -4036,7 +4036,7 @@ def fillna( """ if not isinstance(value, (float, int, str, bool, dict)): raise PySparkTypeError( - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_STRING", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_STR", message_parameters={"arg_name": "value", "arg_type": type(value).__name__}, ) @@ -4221,7 +4221,7 @@ def all_of_(xs: Iterable) -> bool: valid_types = (bool, float, int, str, list, tuple) if not isinstance(to_replace, valid_types + (dict,)): raise PySparkTypeError( - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_LIST_OR_STR_OR_TUPLE", message_parameters={ "arg_name": "to_replace", "arg_type": type(to_replace).__name__, @@ -4248,7 +4248,7 @@ def all_of_(xs: Iterable) -> bool: if not (subset is None or isinstance(subset, (list, tuple, str))): raise PySparkTypeError( - error_class="NOT_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "subset", "arg_type": type(subset).__name__}, ) @@ -4364,7 +4364,7 @@ def approxQuantile( if not isinstance(col, (str, list, tuple)): raise PySparkTypeError( - error_class="NOT_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "col", "arg_type": type(col).__name__}, ) @@ -4405,7 +4405,7 @@ def approxQuantile( if not isinstance(relativeError, (float, int)): raise PySparkTypeError( - error_class="NOT_FLOAT_OR_INTEGER", + error_class="NOT_FLOAT_OR_INT", message_parameters={ "arg_name": "relativeError", "arg_type": type(relativeError).__name__, @@ -4453,12 +4453,12 @@ def corr(self, col1: str, col2: str, method: Optional[str] = None) -> float: """ if not isinstance(col1, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col1", "arg_type": type(col1).__name__}, ) if not isinstance(col2, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col2", "arg_type": type(col2).__name__}, ) if not method: @@ -4501,12 +4501,12 @@ def cov(self, col1: str, col2: str) -> float: """ if not isinstance(col1, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col1", "arg_type": type(col1).__name__}, ) if not isinstance(col2, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col2", "arg_type": type(col2).__name__}, ) return self._jdf.stat().cov(col1, col2) @@ -4554,12 +4554,12 @@ def crosstab(self, col1: str, col2: str) -> "DataFrame": """ if not isinstance(col1, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col1", "arg_type": type(col1).__name__}, ) if not isinstance(col2, str): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col2", "arg_type": type(col2).__name__}, ) return DataFrame(self._jdf.stat().crosstab(col1, col2), self.sparkSession) @@ -4659,7 +4659,7 @@ def withColumns(self, *colsMap: Dict[str, Column]) -> "DataFrame": if not isinstance(colsMap, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "colsMap", "arg_type": type(colsMap).__name__}, ) @@ -4716,7 +4716,7 @@ def withColumn(self, colName: str, col: Column) -> "DataFrame": """ if not isinstance(col, Column): raise PySparkTypeError( - error_class="NOT_A_COLUMN", + error_class="NOT_COLUMN", message_parameters={"arg_name": "col", "arg_type": type(col).__name__}, ) return DataFrame(self._jdf.withColumn(colName, col._jc), self.sparkSession) @@ -4795,7 +4795,7 @@ def withColumnsRenamed(self, colsMap: Dict[str, str]) -> "DataFrame": """ if not isinstance(colsMap, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "colsMap", "arg_type": type(colsMap).__name__}, ) @@ -4827,7 +4827,7 @@ def withMetadata(self, columnName: str, metadata: Dict[str, Any]) -> "DataFrame" """ if not isinstance(metadata, dict): raise PySparkTypeError( - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "metadata", "arg_type": type(metadata).__name__}, ) sc = SparkContext._active_spark_context @@ -4904,7 +4904,7 @@ def drop(self, *cols: "ColumnOrName") -> "DataFrame": # type: ignore[misc] jdf = self._jdf.drop(col._jc) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "col", "arg_type": type(col).__name__}, ) else: @@ -5048,7 +5048,7 @@ def sameSemantics(self, other: "DataFrame") -> bool: """ if not isinstance(other, DataFrame): raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "other", "arg_type": type(other).__name__}, ) return self._jdf.sameSemantics(other._jdf) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 3426f2bdaf6c1..b453154cd56cf 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -3832,7 +3832,7 @@ def when(condition: Column, value: Any) -> Column: # Explicitly not using ColumnOrName type here to make reading condition less opaque if not isinstance(condition, Column): raise PySparkTypeError( - error_class="NOT_A_COLUMN", + error_class="NOT_COLUMN", message_parameters={"arg_name": "condition", "arg_type": type(condition).__name__}, ) v = value._jc if isinstance(value, Column) else value @@ -5470,7 +5470,7 @@ def window( def check_string_field(field, fieldName): # type: ignore[no-untyped-def] if not field or type(field) is not str: raise PySparkTypeError( - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": fieldName, "arg_type": type(field).__name__}, ) @@ -5589,7 +5589,7 @@ def session_window(timeColumn: "ColumnOrName", gapDuration: Union[Column, str]) def check_field(field: Union[Column, str], fieldName: str) -> None: if field is None or not isinstance(field, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": fieldName, "arg_type": type(field).__name__}, ) @@ -5853,7 +5853,7 @@ def assert_true(col: "ColumnOrName", errMsg: Optional[Union[Column, str]] = None return _invoke_function_over_columns("assert_true", col) if not isinstance(errMsg, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": type(errMsg).__name__}, ) @@ -5893,7 +5893,7 @@ def raise_error(errMsg: Union[Column, str]) -> Column: """ if not isinstance(errMsg, (str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": type(errMsg).__name__}, ) @@ -6435,12 +6435,12 @@ def overlay( """ if not isinstance(pos, (int, str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "pos", "arg_type": type(pos).__name__}, ) if len is not None and not isinstance(len, (int, str, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "len", "arg_type": type(len).__name__}, ) @@ -8344,7 +8344,7 @@ def schema_of_json(json: "ColumnOrName", options: Optional[Dict[str, str]] = Non col = _to_java_column(json) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "json", "arg_type": type(json).__name__}, ) @@ -8391,7 +8391,7 @@ def schema_of_csv(csv: "ColumnOrName", options: Optional[Dict[str, str]] = None) col = _to_java_column(csv) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "csv", "arg_type": type(csv).__name__}, ) @@ -9126,7 +9126,7 @@ def from_csv( schema = _to_java_column(schema) else: raise PySparkTypeError( - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "schema", "arg_type": type(schema).__name__}, ) @@ -9937,7 +9937,7 @@ def bucket(numBuckets: Union[Column, int], col: "ColumnOrName") -> Column: """ if not isinstance(numBuckets, (int, Column)): raise PySparkTypeError( - error_class="NOT_COLUMN_OR_INTEGER", + error_class="NOT_COLUMN_OR_INT", message_parameters={"arg_name": "numBuckets", "arg_type": type(numBuckets).__name__}, ) diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py index 0832c9739d6d1..e3e668eb83590 100644 --- a/python/pyspark/sql/tests/connect/test_connect_function.py +++ b/python/pyspark/sql/tests/connect/test_connect_function.py @@ -169,7 +169,7 @@ def test_broadcast(self): self.check_error( exception=pe.exception, - error_class="NOT_A_DATAFRAME", + error_class="NOT_DATAFRAME", message_parameters={"arg_name": "df", "arg_type": "Column"}, ) @@ -370,7 +370,7 @@ def test_when_otherwise(self): self.check_error( exception=pe.exception, - error_class="NOT_A_COLUMN", + error_class="NOT_COLUMN", message_parameters={"arg_name": "condition", "arg_type": "bool"}, ) @@ -1182,7 +1182,7 @@ def test_collection_functions(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "start", "arg_type": "float"}, ) @@ -1191,7 +1191,7 @@ def test_collection_functions(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "length", "arg_type": "float"}, ) @@ -1791,7 +1791,7 @@ def test_json_functions(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_DATATYPE_OR_STRING", + error_class="NOT_COLUMN_OR_DATATYPE_OR_STR", message_parameters={"arg_name": "schema", "arg_type": "list"}, ) @@ -2196,7 +2196,7 @@ def test_time_window_functions(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "slideDuration", "arg_type": "int"}, ) @@ -2205,7 +2205,7 @@ def test_time_window_functions(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "startTime", "arg_type": "int"}, ) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index d0fd39f4e053c..35b12bfd2ec8c 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -122,7 +122,7 @@ def test_with_columns_renamed(self): self.check_error( exception=pe.exception, - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "colsMap", "arg_type": "tuple"}, ) @@ -225,7 +225,7 @@ def test_dropna(self): self.check_error( exception=pe.exception, - error_class="NOT_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "subset", "arg_type": "int"}, ) @@ -305,7 +305,7 @@ def test_fillna(self): self.check_error( exception=pe.exception, - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_STRING", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_STR", message_parameters={"arg_name": "value", "arg_type": "list"}, ) @@ -356,7 +356,7 @@ def test_repartitionByRange_dataframe(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "numPartitions", "arg_type": "list"}, ) @@ -555,7 +555,7 @@ def test_replace(self): self.check_error( exception=pe.exception, - error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INTEGER_OR_LIST_OR_STRING_OR_TUPLE", + error_class="NOT_BOOL_OR_DICT_OR_FLOAT_OR_INT_OR_LIST_OR_STR_OR_TUPLE", message_parameters={"arg_name": "to_replace", "arg_type": "function"}, ) @@ -1489,7 +1489,7 @@ def test_same_semantics_error(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "other", "arg_type": "int"}, ) @@ -1525,7 +1525,7 @@ def test_df_show(self): self.check_error( exception=pe.exception, - error_class="NOT_AN_INTEGER", + error_class="NOT_INT", message_parameters={"arg_name": "n", "arg_type": "bool"}, ) @@ -1534,7 +1534,7 @@ def test_df_show(self): self.check_error( exception=pe.exception, - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={"arg_name": "vertical", "arg_type": "str"}, ) @@ -1543,7 +1543,7 @@ def test_df_show(self): self.check_error( exception=pe.exception, - error_class="NOT_A_BOOLEAN", + error_class="NOT_BOOL", message_parameters={"arg_name": "truncate", "arg_type": "str"}, ) @@ -1615,7 +1615,7 @@ def test_repartition(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "numPartitions", "arg_type": "list"}, ) @@ -1625,7 +1625,7 @@ def test_colregex(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "colName", "arg_type": "int"}, ) @@ -1635,7 +1635,7 @@ def test_where(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "condition", "arg_type": "int"}, ) diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index f8098d185af7f..05492347755e1 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -218,7 +218,7 @@ def test_sampleby(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "col", "arg_type": "int"}, ) @@ -227,7 +227,7 @@ def test_sampleby(self): self.check_error( exception=pe.exception, - error_class="NOT_A_DICT", + error_class="NOT_DICT", message_parameters={"arg_name": "fractions", "arg_type": "list"}, ) @@ -255,7 +255,7 @@ def test_cov(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col1", "arg_type": "int"}, ) @@ -264,7 +264,7 @@ def test_cov(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "col2", "arg_type": "bool"}, ) @@ -764,7 +764,7 @@ def test_overlay(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "pos", "arg_type": "float"}, ) @@ -773,7 +773,7 @@ def test_overlay(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER_OR_STRING", + error_class="NOT_COLUMN_OR_INT_OR_STR", message_parameters={"arg_name": "len", "arg_type": "float"}, ) @@ -1076,7 +1076,7 @@ def test_assert_true(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": "int"}, ) @@ -1096,7 +1096,7 @@ def test_raise_error(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "errMsg", "arg_type": "NoneType"}, ) @@ -1280,7 +1280,7 @@ def test_schema_of_json(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "json", "arg_type": "int"}, ) @@ -1290,7 +1290,7 @@ def test_schema_of_csv(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "csv", "arg_type": "int"}, ) @@ -1301,7 +1301,7 @@ def test_from_csv(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "schema", "arg_type": "int"}, ) @@ -1322,7 +1322,7 @@ def test_when(self): self.check_error( exception=pe.exception, - error_class="NOT_A_COLUMN", + error_class="NOT_COLUMN", message_parameters={"arg_name": "condition", "arg_type": "str"}, ) @@ -1332,7 +1332,7 @@ def test_window(self): self.check_error( exception=pe.exception, - error_class="NOT_A_STRING", + error_class="NOT_STR", message_parameters={"arg_name": "windowDuration", "arg_type": "int"}, ) @@ -1342,7 +1342,7 @@ def test_session_window(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_STRING", + error_class="NOT_COLUMN_OR_STR", message_parameters={"arg_name": "gapDuration", "arg_type": "int"}, ) @@ -1352,7 +1352,7 @@ def test_bucket(self): self.check_error( exception=pe.exception, - error_class="NOT_COLUMN_OR_INTEGER", + error_class="NOT_COLUMN_OR_INT", message_parameters={"arg_name": "numBuckets", "arg_type": "str"}, )