From 72024a5b3f61ecad3f3ef2bf87dd48e8f3f07c06 Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Fri, 27 Dec 2024 23:25:01 +0100 Subject: [PATCH 1/8] remove built in python params --- .../pyspark/sql/connect/functions/builtin.py | 194 +++---- python/pyspark/sql/functions/builtin.py | 544 +++++++++--------- python/pyspark/sql/tests/test_functions.py | 2 +- 3 files changed, 370 insertions(+), 370 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index f13eeab12dd35..f64884e15e6ed 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -339,10 +339,10 @@ def coalesce(*cols: "ColumnOrName") -> Column: coalesce.__doc__ = pysparkfuncs.coalesce.__doc__ -def expr(str: str) -> Column: +def expr(expression: str) -> Column: from pyspark.sql.connect.column import Column as ConnectColumn - return ConnectColumn(SQLExpression(str)) + return ConnectColumn(SQLExpression(expression)) expr.__doc__ = pysparkfuncs.expr.__doc__ @@ -781,13 +781,13 @@ def pmod(dividend: Union["ColumnOrName", float], divisor: Union["ColumnOrName", def width_bucket( v: "ColumnOrName", - min: "ColumnOrName", - max: "ColumnOrName", + minimum: "ColumnOrName", + maximum: "ColumnOrName", numBucket: Union["ColumnOrName", int], ) -> Column: numBucket = _enum_to_value(numBucket) numBucket = lit(numBucket) if isinstance(numBucket, int) else numBucket - return _invoke_function_over_columns("width_bucket", v, min, max, numBucket) + return _invoke_function_over_columns("width_bucket", v, minimum, maximum, numBucket) width_bucket.__doc__ = pysparkfuncs.width_bucket.__doc__ @@ -1003,16 +1003,16 @@ def unhex(col: "ColumnOrName") -> Column: def uniform( - min: Union[Column, int, float], - max: Union[Column, int, float], + minimum: Union[Column, int, float], + maximum: Union[Column, int, float], seed: Optional[Union[Column, int]] = None, ) -> Column: if seed is None: return _invoke_function_over_columns( - "uniform", lit(min), lit(max), lit(random.randint(0, sys.maxsize)) + "uniform", lit(minimum), lit(maximum), lit(random.randint(0, sys.maxsize)) ) else: - return _invoke_function_over_columns("uniform", lit(min), lit(max), lit(seed)) + return _invoke_function_over_columns("uniform", lit(minimum), lit(maximum), lit(seed)) uniform.__doc__ = pysparkfuncs.uniform.__doc__ @@ -2498,29 +2498,29 @@ def encode(col: "ColumnOrName", charset: str) -> Column: encode.__doc__ = pysparkfuncs.encode.__doc__ -def is_valid_utf8(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("is_valid_utf8", _to_col(str)) +def is_valid_utf8(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("is_valid_utf8", _to_col(col)) is_valid_utf8.__doc__ = pysparkfuncs.is_valid_utf8.__doc__ -def make_valid_utf8(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("make_valid_utf8", _to_col(str)) +def make_valid_utf8(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("make_valid_utf8", _to_col(col)) make_valid_utf8.__doc__ = pysparkfuncs.make_valid_utf8.__doc__ -def validate_utf8(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("validate_utf8", _to_col(str)) +def validate_utf8(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("validate_utf8", _to_col(col)) validate_utf8.__doc__ = pysparkfuncs.validate_utf8.__doc__ -def try_validate_utf8(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("try_validate_utf8", _to_col(str)) +def try_validate_utf8(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("try_validate_utf8", _to_col(col)) try_validate_utf8.__doc__ = pysparkfuncs.try_validate_utf8.__doc__ @@ -2533,15 +2533,15 @@ def format_number(col: "ColumnOrName", d: int) -> Column: format_number.__doc__ = pysparkfuncs.format_number.__doc__ -def format_string(format: str, *cols: "ColumnOrName") -> Column: - return _invoke_function("format_string", lit(format), *[_to_col(c) for c in cols]) +def format_string(fmt: str, *cols: "ColumnOrName") -> Column: + return _invoke_function("format_string", lit(fmt), *[_to_col(c) for c in cols]) format_string.__doc__ = pysparkfuncs.format_string.__doc__ -def instr(str: "ColumnOrName", substr: Union[Column, str]) -> Column: - return _invoke_function("instr", _to_col(str), lit(substr)) +def instr(col: "ColumnOrName", substr: Union[Column, str]) -> Column: + return _invoke_function("instr", _to_col(col), lit(substr)) instr.__doc__ = pysparkfuncs.instr.__doc__ @@ -2551,7 +2551,7 @@ def overlay( src: "ColumnOrName", replace: "ColumnOrName", pos: Union["ColumnOrName", int], - len: Union["ColumnOrName", int] = -1, + length: Union["ColumnOrName", int] = -1, ) -> Column: pos = _enum_to_value(pos) if not isinstance(pos, (int, str, Column)): @@ -2559,19 +2559,19 @@ def overlay( errorClass="NOT_COLUMN_OR_INT_OR_STR", messageParameters={"arg_name": "pos", "arg_type": type(pos).__name__}, ) - len = _enum_to_value(len) - if len is not None and not isinstance(len, (int, str, Column)): + length = _enum_to_value(length) + if length is not None and not isinstance(length, (int, str, Column)): raise PySparkTypeError( errorClass="NOT_COLUMN_OR_INT_OR_STR", - messageParameters={"arg_name": "len", "arg_type": type(len).__name__}, + messageParameters={"arg_name": "length", "arg_type": type(length).__name__}, ) if isinstance(pos, int): pos = lit(pos) - if isinstance(len, int): - len = lit(len) + if isinstance(length, int): + length = lit(length) - return _invoke_function_over_columns("overlay", src, replace, pos, len) + return _invoke_function_over_columns("overlay", src, replace, pos, length) overlay.__doc__ = pysparkfuncs.overlay.__doc__ @@ -2592,20 +2592,20 @@ def sentences( def substring( - str: "ColumnOrName", + col: "ColumnOrName", pos: Union["ColumnOrName", int], - len: Union["ColumnOrName", int], + length: Union["ColumnOrName", int], ) -> Column: _pos = lit(pos) if isinstance(pos, int) else _to_col(pos) - _len = lit(len) if isinstance(len, int) else _to_col(len) - return _invoke_function("substring", _to_col(str), _pos, _len) + _len = lit(length) if isinstance(length, int) else _to_col(length) + return _invoke_function("substring", _to_col(col), _pos, _len) substring.__doc__ = pysparkfuncs.substring.__doc__ -def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: - return _invoke_function("substring_index", _to_col(str), lit(delim), lit(count)) +def substring_index(col: "ColumnOrName", delim: str, count: int) -> Column: + return _invoke_function("substring_index", _to_col(col), lit(delim), lit(count)) substring_index.__doc__ = pysparkfuncs.substring_index.__doc__ @@ -2623,8 +2623,8 @@ def levenshtein( levenshtein.__doc__ = pysparkfuncs.levenshtein.__doc__ -def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: - return _invoke_function("locate", lit(substr), _to_col(str), lit(pos)) +def locate(substr: str, col: "ColumnOrName", pos: int = 1) -> Column: + return _invoke_function("locate", lit(substr), _to_col(col), lit(pos)) locate.__doc__ = pysparkfuncs.locate.__doc__ @@ -2632,10 +2632,10 @@ def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: def lpad( col: "ColumnOrName", - len: Union[Column, int], + length: Union[Column, int], pad: Union[Column, str], ) -> Column: - return _invoke_function_over_columns("lpad", col, lit(len), lit(pad)) + return _invoke_function_over_columns("lpad", col, lit(length), lit(pad)) lpad.__doc__ = pysparkfuncs.lpad.__doc__ @@ -2643,10 +2643,10 @@ def lpad( def rpad( col: "ColumnOrName", - len: Union[Column, int], + length: Union[Column, int], pad: Union[Column, str], ) -> Column: - return _invoke_function_over_columns("rpad", col, lit(len), lit(pad)) + return _invoke_function_over_columns("rpad", col, lit(length), lit(pad)) rpad.__doc__ = pysparkfuncs.rpad.__doc__ @@ -2662,34 +2662,34 @@ def repeat(col: "ColumnOrName", n: Union["ColumnOrName", int]) -> Column: def split( - str: "ColumnOrName", + col: "ColumnOrName", pattern: Union[Column, str], limit: Union["ColumnOrName", int] = -1, ) -> Column: limit = _enum_to_value(limit) limit = lit(limit) if isinstance(limit, int) else _to_col(limit) - return _invoke_function("split", _to_col(str), lit(pattern), limit) + return _invoke_function("split", _to_col(col), lit(pattern), limit) split.__doc__ = pysparkfuncs.split.__doc__ -def rlike(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - return _invoke_function_over_columns("rlike", str, regexp) +def rlike(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + return _invoke_function_over_columns("rlike", col, regexp) rlike.__doc__ = pysparkfuncs.rlike.__doc__ -def regexp(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - return _invoke_function_over_columns("regexp", str, regexp) +def regexp(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + return _invoke_function_over_columns("regexp", col, regexp) regexp.__doc__ = pysparkfuncs.regexp.__doc__ -def regexp_like(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - return _invoke_function_over_columns("regexp_like", str, regexp) +def regexp_like(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + return _invoke_function_over_columns("regexp_like", col, regexp) regexp_like.__doc__ = pysparkfuncs.regexp_like.__doc__ @@ -2707,22 +2707,22 @@ def randstr(length: Union[Column, int], seed: Optional[Union[Column, int]] = Non randstr.__doc__ = pysparkfuncs.randstr.__doc__ -def regexp_count(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - return _invoke_function_over_columns("regexp_count", str, regexp) +def regexp_count(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + return _invoke_function_over_columns("regexp_count", col, regexp) regexp_count.__doc__ = pysparkfuncs.regexp_count.__doc__ -def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: - return _invoke_function("regexp_extract", _to_col(str), lit(pattern), lit(idx)) +def regexp_extract(col: "ColumnOrName", pattern: str, idx: int) -> Column: + return _invoke_function("regexp_extract", _to_col(col), lit(pattern), lit(idx)) regexp_extract.__doc__ = pysparkfuncs.regexp_extract.__doc__ def regexp_extract_all( - str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None + col: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: if idx is None: return _invoke_function_over_columns("regexp_extract_all", str, regexp) @@ -2742,20 +2742,20 @@ def regexp_replace( regexp_replace.__doc__ = pysparkfuncs.regexp_replace.__doc__ -def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - return _invoke_function_over_columns("regexp_substr", str, regexp) +def regexp_substr(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + return _invoke_function_over_columns("regexp_substr", col, regexp) regexp_substr.__doc__ = pysparkfuncs.regexp_substr.__doc__ def regexp_instr( - str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None + col: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: if idx is None: - return _invoke_function_over_columns("regexp_instr", str, regexp) + return _invoke_function_over_columns("regexp_instr", col, regexp) else: - return _invoke_function_over_columns("regexp_instr", str, regexp, lit(idx)) + return _invoke_function_over_columns("regexp_instr", col, regexp, lit(idx)) regexp_instr.__doc__ = pysparkfuncs.regexp_instr.__doc__ @@ -2854,12 +2854,12 @@ def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnO def substr( - str: "ColumnOrName", pos: "ColumnOrName", len: Optional["ColumnOrName"] = None + col: "ColumnOrName", pos: "ColumnOrName", length: Optional["ColumnOrName"] = None ) -> Column: if len is not None: - return _invoke_function_over_columns("substr", str, pos, len) + return _invoke_function_over_columns("substr", col, pos, length) else: - return _invoke_function_over_columns("substr", str, pos) + return _invoke_function_over_columns("substr", col, pos) substr.__doc__ = pysparkfuncs.substr.__doc__ @@ -2896,48 +2896,48 @@ def printf(format: "ColumnOrName", *cols: "ColumnOrName") -> Column: printf.__doc__ = pysparkfuncs.printf.__doc__ -def url_decode(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("url_decode", str) +def url_decode(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("url_decode", col) url_decode.__doc__ = pysparkfuncs.url_decode.__doc__ -def try_url_decode(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("try_url_decode", str) +def try_url_decode(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("try_url_decode", col) try_url_decode.__doc__ = pysparkfuncs.try_url_decode.__doc__ -def url_encode(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("url_encode", str) +def url_encode(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("url_encode", col) url_encode.__doc__ = pysparkfuncs.url_encode.__doc__ def position( - substr: "ColumnOrName", str: "ColumnOrName", start: Optional["ColumnOrName"] = None + substr: "ColumnOrName", col: "ColumnOrName", start: Optional["ColumnOrName"] = None ) -> Column: if start is not None: - return _invoke_function_over_columns("position", substr, str, start) + return _invoke_function_over_columns("position", substr, col, start) else: - return _invoke_function_over_columns("position", substr, str) + return _invoke_function_over_columns("position", substr, col) position.__doc__ = pysparkfuncs.position.__doc__ -def endswith(str: "ColumnOrName", suffix: "ColumnOrName") -> Column: - return _invoke_function_over_columns("endswith", str, suffix) +def endswith(col: "ColumnOrName", suffix: "ColumnOrName") -> Column: + return _invoke_function_over_columns("endswith", col, suffix) endswith.__doc__ = pysparkfuncs.endswith.__doc__ -def startswith(str: "ColumnOrName", prefix: "ColumnOrName") -> Column: - return _invoke_function_over_columns("startswith", str, prefix) +def startswith(col: "ColumnOrName", prefix: "ColumnOrName") -> Column: + return _invoke_function_over_columns("startswith", col, prefix) startswith.__doc__ = pysparkfuncs.startswith.__doc__ @@ -2967,25 +2967,25 @@ def try_to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: try_to_number.__doc__ = pysparkfuncs.try_to_number.__doc__ -def btrim(str: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: +def btrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: if trim is not None: - return _invoke_function_over_columns("btrim", str, trim) + return _invoke_function_over_columns("btrim", col, trim) else: - return _invoke_function_over_columns("btrim", str) + return _invoke_function_over_columns("btrim", col) btrim.__doc__ = pysparkfuncs.btrim.__doc__ -def char_length(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("char_length", str) +def char_length(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("char_length", col) char_length.__doc__ = pysparkfuncs.char_length.__doc__ -def character_length(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("character_length", str) +def character_length(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("character_length", col) character_length.__doc__ = pysparkfuncs.character_length.__doc__ @@ -3005,60 +3005,60 @@ def elt(*inputs: "ColumnOrName") -> Column: elt.__doc__ = pysparkfuncs.elt.__doc__ -def find_in_set(str: "ColumnOrName", str_array: "ColumnOrName") -> Column: - return _invoke_function_over_columns("find_in_set", str, str_array) +def find_in_set(col: "ColumnOrName", str_array: "ColumnOrName") -> Column: + return _invoke_function_over_columns("find_in_set", col, str_array) find_in_set.__doc__ = pysparkfuncs.find_in_set.__doc__ def like( - str: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None + col: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None ) -> Column: if escapeChar is not None: - return _invoke_function_over_columns("like", str, pattern, escapeChar) + return _invoke_function_over_columns("like", col, pattern, escapeChar) else: - return _invoke_function_over_columns("like", str, pattern) + return _invoke_function_over_columns("like", col, pattern) like.__doc__ = pysparkfuncs.like.__doc__ def ilike( - str: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None + col: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None ) -> Column: if escapeChar is not None: - return _invoke_function_over_columns("ilike", str, pattern, escapeChar) + return _invoke_function_over_columns("ilike", col, pattern, escapeChar) else: - return _invoke_function_over_columns("ilike", str, pattern) + return _invoke_function_over_columns("ilike", col, pattern) ilike.__doc__ = pysparkfuncs.ilike.__doc__ -def lcase(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("lcase", str) +def lcase(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("lcase", col) lcase.__doc__ = pysparkfuncs.lcase.__doc__ -def ucase(str: "ColumnOrName") -> Column: - return _invoke_function_over_columns("ucase", str) +def ucase(col: "ColumnOrName") -> Column: + return _invoke_function_over_columns("ucase", col) ucase.__doc__ = pysparkfuncs.ucase.__doc__ -def left(str: "ColumnOrName", len: "ColumnOrName") -> Column: - return _invoke_function_over_columns("left", str, len) +def left(col: "ColumnOrName", length: "ColumnOrName") -> Column: + return _invoke_function_over_columns("left", col, length) left.__doc__ = pysparkfuncs.left.__doc__ -def right(str: "ColumnOrName", len: "ColumnOrName") -> Column: - return _invoke_function_over_columns("right", str, len) +def right(col: "ColumnOrName", length: "ColumnOrName") -> Column: + return _invoke_function_over_columns("right", col, length) right.__doc__ = pysparkfuncs.right.__doc__ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 7b14598a0ef4e..a2f6ede6d7142 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6285,8 +6285,8 @@ def pmod(dividend: Union["ColumnOrName", float], divisor: Union["ColumnOrName", @_try_remote_functions def width_bucket( v: "ColumnOrName", - min: "ColumnOrName", - max: "ColumnOrName", + minimum: "ColumnOrName", + maximum: "ColumnOrName", numBucket: Union["ColumnOrName", int], ) -> Column: """ @@ -6300,9 +6300,9 @@ def width_bucket( ---------- v : :class:`~pyspark.sql.Column` or column name value to compute a bucket number in the histogram - min : :class:`~pyspark.sql.Column` or column name + minimum : :class:`~pyspark.sql.Column` or column name minimum value of the histogram - max : :class:`~pyspark.sql.Column` or column name + maximum : :class:`~pyspark.sql.Column` or column name maximum value of the histogram numBucket : :class:`~pyspark.sql.Column`, column name or int the number of buckets @@ -6320,20 +6320,20 @@ def width_bucket( ... (-2.1, 1.3, 3.4, 3), ... (8.1, 0.0, 5.7, 4), ... (-0.9, 5.2, 0.5, 2)], - ... ['v', 'min', 'max', 'n']) - >>> df.select("*", sf.width_bucket('v', 'min', 'max', 'n')).show() - +----+---+----+---+----------------------------+ - | v|min| max| n|width_bucket(v, min, max, n)| - +----+---+----+---+----------------------------+ - | 5.3|0.2|10.6| 5| 3| - |-2.1|1.3| 3.4| 3| 0| - | 8.1|0.0| 5.7| 4| 5| - |-0.9|5.2| 0.5| 2| 3| - +----+---+----+---+----------------------------+ + ... ['v', 'minimum', 'maximum', 'n']) + >>> df.select("*", sf.width_bucket('v', 'minimum', 'maximum', 'n')).show() + +----+-------+--------+---+------------------------------------+ + | v|minimum| maximum| n|width_bucket(v, minimum, maximum, n)| + +----+-------+--------+---+------------------------------------+ + | 5.3| 0.2|10.6 | 5| 3| + |-2.1| 1.3| 3.4 | 3| 0| + | 8.1| 0.0| 5.7 | 4| 5| + |-0.9| 5.2| 0.5 | 2| 3| + +----+-------+--------+---+------------------------------------+ """ numBucket = _enum_to_value(numBucket) numBucket = lit(numBucket) if isinstance(numBucket, int) else numBucket - return _invoke_function_over_columns("width_bucket", v, min, max, numBucket) + return _invoke_function_over_columns("width_bucket", v, minimum, maximum, numBucket) @_try_remote_functions @@ -8063,7 +8063,7 @@ def spark_partition_id() -> Column: @_try_remote_functions -def expr(str: str) -> Column: +def expr(expression: str) -> Column: """Parses the expression string into the column that it represents .. versionadded:: 1.5.0 @@ -8073,7 +8073,7 @@ def expr(str: str) -> Column: Parameters ---------- - str : expression string + expression : expression string expression defined in string. Returns @@ -8093,7 +8093,7 @@ def expr(str: str) -> Column: | Bob| 3| +-----+------------+ """ - return _invoke_function("expr", str) + return _invoke_function("expr", expression) @overload @@ -9389,7 +9389,7 @@ def localtimestamp() -> Column: @_try_remote_functions -def date_format(date: "ColumnOrName", format: str) -> Column: +def date_format(date: "ColumnOrName", fmt: str) -> Column: """ Converts a date/timestamp/string to a value of string in the format specified by the date format given by the second argument. @@ -9412,7 +9412,7 @@ def date_format(date: "ColumnOrName", format: str) -> Column: ---------- date : :class:`~pyspark.sql.Column` or column name input column of values to format. - format: literal string + fmt: literal string format to use to represent datetime values. See Also @@ -9487,7 +9487,7 @@ def date_format(date: "ColumnOrName", format: str) -> Column: """ from pyspark.sql.classic.column import _to_java_column - return _invoke_function("date_format", _to_java_column(date), _enum_to_value(format)) + return _invoke_function("date_format", _to_java_column(date), _enum_to_value(fmt)) @_try_remote_functions @@ -11279,7 +11279,7 @@ def months_between(date1: "ColumnOrName", date2: "ColumnOrName", roundOff: bool @_try_remote_functions -def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: +def to_date(col: "ColumnOrName", fmt: Optional[str] = None) -> Column: """Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.DateType` using the optionally specified format. Specify formats according to `datetime pattern`_. By default, it follows casting rules to :class:`pyspark.sql.types.DateType` if the format @@ -11296,7 +11296,7 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: ---------- col : :class:`~pyspark.sql.Column` or column name input column of values to convert. - format: literal string, optional + fmt: literal string, optional format to use to convert date values. Returns @@ -11333,10 +11333,10 @@ def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: """ from pyspark.sql.classic.column import _to_java_column - if format is None: + if fmt is None: return _invoke_function_over_columns("to_date", col) else: - return _invoke_function("to_date", _to_java_column(col), _enum_to_value(format)) + return _invoke_function("to_date", _to_java_column(col), _enum_to_value(fmt)) @_try_remote_functions @@ -11516,12 +11516,12 @@ def to_timestamp(col: "ColumnOrName") -> Column: @overload -def to_timestamp(col: "ColumnOrName", format: str) -> Column: +def to_timestamp(col: "ColumnOrName", fmt: str) -> Column: ... @_try_remote_functions -def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: +def to_timestamp(col: "ColumnOrName", fmt: Optional[str] = None) -> Column: """Converts a :class:`~pyspark.sql.Column` into :class:`pyspark.sql.types.TimestampType` using the optionally specified format. Specify formats according to `datetime pattern`_. By default, it follows casting rules to :class:`pyspark.sql.types.TimestampType` if the format @@ -11538,7 +11538,7 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: ---------- col : :class:`~pyspark.sql.Column` or column name column values to convert. - format: literal string, optional + fmt: literal string, optional format to use to convert timestamp values. Returns @@ -11582,16 +11582,16 @@ def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: """ from pyspark.sql.classic.column import _to_java_column - if format is None: + if fmt is None: return _invoke_function_over_columns("to_timestamp", col) else: - return _invoke_function("to_timestamp", _to_java_column(col), _enum_to_value(format)) + return _invoke_function("to_timestamp", _to_java_column(col), _enum_to_value(fmt)) @_try_remote_functions -def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: +def try_to_timestamp(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: """ - Parses the `col` with the `format` to a timestamp. The function always + Parses the `col` with the `fmt` to a timestamp. The function always returns null on an invalid input with/without ANSI SQL mode enabled. The result data type is consistent with the value of configuration `spark.sql.timestampType`. @@ -11601,7 +11601,7 @@ def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = Non ---------- col : :class:`~pyspark.sql.Column` or column name column values to convert. - format: literal string, optional + fmt: literal string, optional format to use to convert timestamp values. See Also @@ -11651,8 +11651,8 @@ def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = Non | NULL| +-------------------+ """ - if format is not None: - return _invoke_function_over_columns("try_to_timestamp", col, format) + if fmt is not None: + return _invoke_function_over_columns("try_to_timestamp", col, fmt) else: return _invoke_function_over_columns("try_to_timestamp", col) @@ -11815,7 +11815,7 @@ def xpath_string(xml: "ColumnOrName", path: "ColumnOrName") -> Column: @_try_remote_functions -def trunc(date: "ColumnOrName", format: str) -> Column: +def trunc(date: "ColumnOrName", fmt: str) -> Column: """ Returns date truncated to the unit specified by the format. @@ -11862,11 +11862,11 @@ def trunc(date: "ColumnOrName", format: str) -> Column: """ from pyspark.sql.classic.column import _to_java_column - return _invoke_function("trunc", _to_java_column(date), _enum_to_value(format)) + return _invoke_function("trunc", _to_java_column(date), _enum_to_value(fmt)) @_try_remote_functions -def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: +def date_trunc(fmt: str, timestamp: "ColumnOrName") -> Column: """ Returns timestamp truncated to the unit specified by the format. @@ -11877,7 +11877,7 @@ def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: Parameters ---------- - format : literal string + fmt : literal string 'year', 'yyyy', 'yy' to truncate by year, 'month', 'mon', 'mm' to truncate by month, 'day', 'dd' to truncate by day, @@ -11915,7 +11915,7 @@ def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: """ from pyspark.sql.classic.column import _to_java_column - return _invoke_function("date_trunc", _enum_to_value(format), _to_java_column(timestamp)) + return _invoke_function("date_trunc", _enum_to_value(fmt), _to_java_column(timestamp)) @_try_remote_functions @@ -12009,7 +12009,7 @@ def last_day(date: "ColumnOrName") -> Column: @_try_remote_functions -def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss") -> Column: +def from_unixtime(timestamp: "ColumnOrName", fmt: str = "yyyy-MM-dd HH:mm:ss") -> Column: """ Converts the number of seconds from unix epoch (1970-01-01 00:00:00 UTC) to a string representing the timestamp of that moment in the current system time zone in the given @@ -12024,7 +12024,7 @@ def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss" ---------- timestamp : :class:`~pyspark.sql.Column` or column name column of unix time values. - format : literal string, optional + fmt : literal string, optional format to use to convert to (default: yyyy-MM-dd HH:mm:ss) Returns @@ -12054,11 +12054,11 @@ def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss" """ from pyspark.sql.classic.column import _to_java_column - return _invoke_function("from_unixtime", _to_java_column(timestamp), _enum_to_value(format)) + return _invoke_function("from_unixtime", _to_java_column(timestamp), _enum_to_value(fmt)) @overload -def unix_timestamp(timestamp: "ColumnOrName", format: str = ...) -> Column: +def unix_timestamp(timestamp: "ColumnOrName", fmt: str = ...) -> Column: ... @@ -12069,7 +12069,7 @@ def unix_timestamp() -> Column: @_try_remote_functions def unix_timestamp( - timestamp: Optional["ColumnOrName"] = None, format: str = "yyyy-MM-dd HH:mm:ss" + timestamp: Optional["ColumnOrName"] = None, fmt: str = "yyyy-MM-dd HH:mm:ss" ) -> Column: """ Convert time string with given pattern ('yyyy-MM-dd HH:mm:ss', by default) @@ -12087,7 +12087,7 @@ def unix_timestamp( ---------- timestamp : :class:`~pyspark.sql.Column` or column name, optional timestamps of string values. - format : literal string, optional + fmt : literal string, optional alternative format to use for converting (default: yyyy-MM-dd HH:mm:ss). Returns @@ -12137,7 +12137,7 @@ def unix_timestamp( if timestamp is None: return _invoke_function("unix_timestamp") - return _invoke_function("unix_timestamp", _to_java_column(timestamp), _enum_to_value(format)) + return _invoke_function("unix_timestamp", _to_java_column(timestamp), _enum_to_value(fmt)) @_try_remote_functions @@ -12793,7 +12793,7 @@ def check_field(field: Union[Column, str], fieldName: str) -> None: @_try_remote_functions def to_unix_timestamp( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: """ Returns the UNIX timestamp of the given time. @@ -12804,7 +12804,7 @@ def to_unix_timestamp( ---------- timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or column name, optional + fmt : :class:`~pyspark.sql.Column` or column name, optional format to use to convert UNIX timestamp values. See Also @@ -12856,8 +12856,8 @@ def to_unix_timestamp( >>> spark.conf.unset("spark.sql.session.timeZone") """ - if format is not None: - return _invoke_function_over_columns("to_unix_timestamp", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_unix_timestamp", timestamp, fmt) else: return _invoke_function_over_columns("to_unix_timestamp", timestamp) @@ -12865,10 +12865,10 @@ def to_unix_timestamp( @_try_remote_functions def to_timestamp_ltz( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: """ - Parses the `timestamp` with the `format` to a timestamp with time zone. + Parses the `timestamp` with the `fmt` to a timestamp with time zone. Returns null with invalid input. .. versionadded:: 3.5.0 @@ -12877,7 +12877,7 @@ def to_timestamp_ltz( ---------- timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or column name, optional + fmtr : :class:`~pyspark.sql.Column` or column name, optional format to use to convert type `TimestampType` timestamp values. See Also @@ -12926,8 +12926,8 @@ def to_timestamp_ltz( |2025+01+09|yyyy+MM+dd| 2025-01-09 00:00:00| +----------+----------+-------------------------+ """ - if format is not None: - return _invoke_function_over_columns("to_timestamp_ltz", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_timestamp_ltz", timestamp, fmt) else: return _invoke_function_over_columns("to_timestamp_ltz", timestamp) @@ -12935,10 +12935,10 @@ def to_timestamp_ltz( @_try_remote_functions def to_timestamp_ntz( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: """ - Parses the `timestamp` with the `format` to a timestamp without time zone. + Parses the `timestamp` with the `fmt` to a timestamp without time zone. Returns null with invalid input. .. versionadded:: 3.5.0 @@ -12947,7 +12947,7 @@ def to_timestamp_ntz( ---------- timestamp : :class:`~pyspark.sql.Column` or column name Input column or strings. - format : :class:`~pyspark.sql.Column` or column name, optional + fmt : :class:`~pyspark.sql.Column` or column name, optional format to use to convert type `TimestampNTZType` timestamp values. See Also @@ -12996,8 +12996,8 @@ def to_timestamp_ntz( |2025+01+09|yyyy+MM+dd| 2025-01-09 00:00:00| +----------+----------+-------------------------+ """ - if format is not None: - return _invoke_function_over_columns("to_timestamp_ntz", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_timestamp_ntz", timestamp, fmt) else: return _invoke_function_over_columns("to_timestamp_ntz", timestamp) @@ -14076,7 +14076,7 @@ def encode(col: "ColumnOrName", charset: str) -> Column: @_try_remote_functions -def is_valid_utf8(str: "ColumnOrName") -> Column: +def is_valid_utf8(col: "ColumnOrName") -> Column: """ Returns true if the input is a valid UTF-8 string, otherwise returns false. @@ -14084,7 +14084,7 @@ def is_valid_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -14108,11 +14108,11 @@ def is_valid_utf8(str: "ColumnOrName") -> Column: | true| +-----------------------+ """ - return _invoke_function_over_columns("is_valid_utf8", str) + return _invoke_function_over_columns("is_valid_utf8", col) @_try_remote_functions -def make_valid_utf8(str: "ColumnOrName") -> Column: +def make_valid_utf8(col: "ColumnOrName") -> Column: """ Returns a new string in which all invalid UTF-8 byte sequences, if any, are replaced by the Unicode replacement character (U+FFFD). @@ -14145,11 +14145,11 @@ def make_valid_utf8(str: "ColumnOrName") -> Column: | SparkSQL| +-------------------------+ """ - return _invoke_function_over_columns("make_valid_utf8", str) + return _invoke_function_over_columns("make_valid_utf8", col) @_try_remote_functions -def validate_utf8(str: "ColumnOrName") -> Column: +def validate_utf8(col: "ColumnOrName") -> Column: """ Returns the input value if it corresponds to a valid UTF-8 string, or emits an error otherwise. @@ -14181,11 +14181,11 @@ def validate_utf8(str: "ColumnOrName") -> Column: | SparkSQL| +-----------------------+ """ - return _invoke_function_over_columns("validate_utf8", str) + return _invoke_function_over_columns("validate_utf8", col) @_try_remote_functions -def try_validate_utf8(str: "ColumnOrName") -> Column: +def try_validate_utf8(col: "ColumnOrName") -> Column: """ Returns the input value if it corresponds to a valid UTF-8 string, or NULL otherwise. @@ -14193,7 +14193,7 @@ def try_validate_utf8(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name A column of strings, each representing a UTF-8 byte sequence. Returns @@ -14217,7 +14217,7 @@ def try_validate_utf8(str: "ColumnOrName") -> Column: | SparkSQL| +---------------------------+ """ - return _invoke_function_over_columns("try_validate_utf8", str) + return _invoke_function_over_columns("try_validate_utf8", col) @_try_remote_functions @@ -14260,7 +14260,7 @@ def format_number(col: "ColumnOrName", d: int) -> Column: @_try_remote_functions -def format_string(format: str, *cols: "ColumnOrName") -> Column: +def format_string(fmt: str, *cols: "ColumnOrName") -> Column: """ Formats the arguments in printf-style and returns the result as a string column. @@ -14271,7 +14271,7 @@ def format_string(format: str, *cols: "ColumnOrName") -> Column: Parameters ---------- - format : literal string + fmt : literal string string that can contain embedded format tags and used as result column's value cols : :class:`~pyspark.sql.Column` or column name column names or :class:`~pyspark.sql.Column`\\s to be used in formatting @@ -14300,12 +14300,12 @@ def format_string(format: str, *cols: "ColumnOrName") -> Column: sc = _get_active_spark_context() return _invoke_function( - "format_string", _enum_to_value(format), _to_seq(sc, cols, _to_java_column) + "format_string", _enum_to_value(fmt), _to_seq(sc, cols, _to_java_column) ) @_try_remote_functions -def instr(str: "ColumnOrName", substr: Union[Column, str]) -> Column: +def instr(col: "ColumnOrName", substr: Union[Column, str]) -> Column: """ Locate the position of the first occurrence of substr column in the given string. Returns null if either of the arguments are null. @@ -14322,7 +14322,7 @@ def instr(str: "ColumnOrName", substr: Union[Column, str]) -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name target column to work on. substr : :class:`~pyspark.sql.Column` or literal string substring to look for. @@ -14368,7 +14368,7 @@ def instr(str: "ColumnOrName", substr: Union[Column, str]) -> Column: | xyz| 0| +----+---------------------------+ """ - return _invoke_function_over_columns("instr", str, lit(substr)) + return _invoke_function_over_columns("instr", col, lit(substr)) @_try_remote_functions @@ -14376,11 +14376,11 @@ def overlay( src: "ColumnOrName", replace: "ColumnOrName", pos: Union["ColumnOrName", int], - len: Union["ColumnOrName", int] = -1, + length: Union["ColumnOrName", int] = -1, ) -> Column: """ Overlay the specified portion of `src` with `replace`, - starting from byte position `pos` of `src` and proceeding for `len` bytes. + starting from byte position `pos` of `src` and proceeding for `length` bytes. .. versionadded:: 3.0.0 @@ -14395,7 +14395,7 @@ def overlay( the substitution string pos : :class:`~pyspark.sql.Column` or column name or int the starting position in src - len : :class:`~pyspark.sql.Column` or column name or int, optional + length : :class:`~pyspark.sql.Column` or column name or int, optional the number of bytes to replace in src string by 'replace' defaults to -1, which represents the length of the 'replace' string @@ -14435,19 +14435,19 @@ def overlay( errorClass="NOT_COLUMN_OR_INT_OR_STR", messageParameters={"arg_name": "pos", "arg_type": type(pos).__name__}, ) - len = _enum_to_value(len) - if len is not None and not isinstance(len, (int, str, Column)): + length = _enum_to_value(length) + if length is not None and not isinstance(length, (int, str, Column)): raise PySparkTypeError( errorClass="NOT_COLUMN_OR_INT_OR_STR", - messageParameters={"arg_name": "len", "arg_type": type(len).__name__}, + messageParameters={"arg_name": "length", "arg_type": type(length).__name__}, ) if isinstance(pos, int): pos = lit(pos) - if isinstance(len, int): - len = lit(len) + if isinstance(length, int): + length = lit(length) - return _invoke_function_over_columns("overlay", src, replace, pos, len) + return _invoke_function_over_columns("overlay", src, replace, pos, length) @_try_remote_functions @@ -14533,13 +14533,13 @@ def sentences( @_try_remote_functions def substring( - str: "ColumnOrName", + col: "ColumnOrName", pos: Union["ColumnOrName", int], - len: Union["ColumnOrName", int], + length: Union["ColumnOrName", int], ) -> Column: """ - Substring starts at `pos` and is of length `len` when str is String type or - returns the slice of byte array that starts at `pos` in byte and is of length `len` + Substring starts at `pos` and is of length `length` when str is String type or + returns the slice of byte array that starts at `pos` in byte and is of length `length` when str is Binary type. .. versionadded:: 1.5.0 @@ -14553,7 +14553,7 @@ def substring( Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name target column to work on. pos : :class:`~pyspark.sql.Column` or column name or int starting position in str. @@ -14561,11 +14561,11 @@ def substring( .. versionchanged:: 4.0.0 `pos` now accepts column and column name. - len : :class:`~pyspark.sql.Column` or column name or int + length : :class:`~pyspark.sql.Column` or column name or int length of chars. .. versionchanged:: 4.0.0 - `len` now accepts column and column name. + `length` now accepts column and column name. Returns ------- @@ -14638,13 +14638,13 @@ def substring( """ pos = _enum_to_value(pos) pos = lit(pos) if isinstance(pos, int) else pos - len = _enum_to_value(len) - len = lit(len) if isinstance(len, int) else len - return _invoke_function_over_columns("substring", str, pos, len) + length = _enum_to_value(length) + length = lit(length) if isinstance(length, int) else length + return _invoke_function_over_columns("substring", col, pos, length) @_try_remote_functions -def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: +def substring_index(col: "ColumnOrName", delim: str, count: int) -> Column: """ Returns the substring from string str before count occurrences of the delimiter delim. If count is positive, everything the left of the final delimiter (counting from left) is @@ -14658,7 +14658,7 @@ def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name target column to work on. delim : literal string delimiter of values. @@ -14699,7 +14699,7 @@ def substring_index(str: "ColumnOrName", delim: str, count: int) -> Column: from pyspark.sql.classic.column import _to_java_column return _invoke_function( - "substring_index", _to_java_column(str), _enum_to_value(delim), _enum_to_value(count) + "substring_index", _to_java_column(col), _enum_to_value(delim), _enum_to_value(count) ) @@ -14760,7 +14760,7 @@ def levenshtein( @_try_remote_functions -def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: +def locate(substr: str, col: "ColumnOrName", pos: int = 1) -> Column: """ Locate the position of the first occurrence of substr in a string column, after position pos. @@ -14773,7 +14773,7 @@ def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: ---------- substr : literal string a string - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name a Column of :class:`pyspark.sql.types.StringType` pos : int, optional start position (zero based) @@ -14817,18 +14817,18 @@ def locate(substr: str, str: "ColumnOrName", pos: int = 1) -> Column: from pyspark.sql.classic.column import _to_java_column return _invoke_function( - "locate", _enum_to_value(substr), _to_java_column(str), _enum_to_value(pos) + "locate", _enum_to_value(substr), _to_java_column(col), _enum_to_value(pos) ) @_try_remote_functions def lpad( col: "ColumnOrName", - len: Union[Column, int], + length: Union[Column, int], pad: Union[Column, str], ) -> Column: """ - Left-pad the string column to width `len` with `pad`. + Left-pad the string column to width `length` with `pad`. .. versionadded:: 1.5.0 @@ -14839,7 +14839,7 @@ def lpad( ---------- col : :class:`~pyspark.sql.Column` or column name target column to work on. - len : :class:`~pyspark.sql.Column` or int + length : :class:`~pyspark.sql.Column` or int length of the final string. .. versionchanged:: 4.0.0 @@ -14888,17 +14888,17 @@ def lpad( | 12| uvuv12| +----+-------------------+ """ - return _invoke_function_over_columns("lpad", col, lit(len), lit(pad)) + return _invoke_function_over_columns("lpad", col, lit(length), lit(pad)) @_try_remote_functions def rpad( col: "ColumnOrName", - len: Union[Column, int], + length: Union[Column, int], pad: Union[Column, str], ) -> Column: """ - Right-pad the string column to width `len` with `pad`. + Right-pad the string column to width `length` with `pad`. .. versionadded:: 1.5.0 @@ -14909,7 +14909,7 @@ def rpad( ---------- col : :class:`~pyspark.sql.Column` or str target column to work on. - len : :class:`~pyspark.sql.Column` or int + length : :class:`~pyspark.sql.Column` or int length of the final string. .. versionchanged:: 4.0.0 @@ -14958,7 +14958,7 @@ def rpad( | 12| 12uvuv| +----+-------------------+ """ - return _invoke_function_over_columns("rpad", col, lit(len), lit(pad)) + return _invoke_function_over_columns("rpad", col, lit(length), lit(pad)) @_try_remote_functions @@ -15025,7 +15025,7 @@ def repeat(col: "ColumnOrName", n: Union["ColumnOrName", int]) -> Column: @_try_remote_functions def split( - str: "ColumnOrName", + col: "ColumnOrName", pattern: Union[Column, str], limit: Union["ColumnOrName", int] = -1, ) -> Column: @@ -15039,7 +15039,7 @@ def split( Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name a string expression to split pattern : :class:`~pyspark.sql.Column` or literal string a string representing a regular expression. The regex string should be @@ -15126,18 +15126,18 @@ def split( """ limit = _enum_to_value(limit) limit = lit(limit) if isinstance(limit, int) else limit - return _invoke_function_over_columns("split", str, lit(pattern), limit) + return _invoke_function_over_columns("split", col, lit(pattern), limit) @_try_remote_functions -def rlike(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns true if `str` matches the Java regex `regexp`, or false otherwise. +def rlike(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + r"""Returns true if `col` matches the Java regex `regexp`, or false otherwise. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15145,7 +15145,7 @@ def rlike(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Returns ------- :class:`~pyspark.sql.Column` - true if `str` matches a Java regex, or false otherwise. + true if `col` matches a Java regex, or false otherwise. Examples -------- @@ -15157,18 +15157,18 @@ def rlike(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: >>> df.select(rlike("str", col("regexp")).alias('d')).collect() [Row(d=True)] """ - return _invoke_function_over_columns("rlike", str, regexp) + return _invoke_function_over_columns("rlike", col, regexp) @_try_remote_functions -def regexp(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns true if `str` matches the Java regex `regexp`, or false otherwise. +def regexp(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + r"""Returns true if `col` matches the Java regex `regexp`, or false otherwise. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15176,7 +15176,7 @@ def regexp(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Returns ------- :class:`~pyspark.sql.Column` - true if `str` matches a Java regex, or false otherwise. + true if `col` matches a Java regex, or false otherwise. Examples -------- @@ -15210,18 +15210,18 @@ def regexp(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: | true| +-------------------+ """ - return _invoke_function_over_columns("regexp", str, regexp) + return _invoke_function_over_columns("regexp", col, regexp) @_try_remote_functions -def regexp_like(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns true if `str` matches the Java regex `regexp`, or false otherwise. +def regexp_like(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + r"""Returns true if `col` matches the Java regex `regexp`, or false otherwise. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15229,7 +15229,7 @@ def regexp_like(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Returns ------- :class:`~pyspark.sql.Column` - true if `str` matches a Java regex, or false otherwise. + true if `col` matches a Java regex, or false otherwise. Examples -------- @@ -15263,7 +15263,7 @@ def regexp_like(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: | true| +------------------------+ """ - return _invoke_function_over_columns("regexp_like", str, regexp) + return _invoke_function_over_columns("regexp_like", col, regexp) @_try_remote_functions @@ -15321,15 +15321,15 @@ def randstr(length: Union[Column, int], seed: Optional[Union[Column, int]] = Non @_try_remote_functions -def regexp_count(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: +def regexp_count(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: r"""Returns a count of the number of times that the Java regex pattern `regexp` is matched - in the string `str`. + in the string `col`. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15349,11 +15349,11 @@ def regexp_count(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: >>> df.select(regexp_count("str", col("regexp")).alias('d')).collect() [Row(d=3)] """ - return _invoke_function_over_columns("regexp_count", str, regexp) + return _invoke_function_over_columns("regexp_count", col, regexp) @_try_remote_functions -def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: +def regexp_extract(col: "ColumnOrName", pattern: str, idx: int) -> Column: r"""Extract a specific group matched by the Java regex `regexp`, from the specified string column. If the regex did not match, or the specified group did not match, an empty string is returned. @@ -15364,7 +15364,7 @@ def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. pattern : str regex pattern to apply. @@ -15391,22 +15391,22 @@ def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: from pyspark.sql.classic.column import _to_java_column return _invoke_function( - "regexp_extract", _to_java_column(str), _enum_to_value(pattern), _enum_to_value(idx) + "regexp_extract", _to_java_column(col), _enum_to_value(pattern), _enum_to_value(idx) ) @_try_remote_functions def regexp_extract_all( - str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None + col: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: - r"""Extract all strings in the `str` that match the Java regex `regexp` + r"""Extract all strings in the `col` that match the Java regex `regexp` and corresponding to the regex group index. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15416,7 +15416,7 @@ def regexp_extract_all( Returns ------- :class:`~pyspark.sql.Column` - all strings in the `str` that match a Java regex and corresponding to the regex group index. + all strings in the `col` that match a Java regex and corresponding to the regex group index. Examples -------- @@ -15431,14 +15431,14 @@ def regexp_extract_all( [Row(d=['100', '300'])] """ if idx is None: - return _invoke_function_over_columns("regexp_extract_all", str, regexp) + return _invoke_function_over_columns("regexp_extract_all", col, regexp) else: - return _invoke_function_over_columns("regexp_extract_all", str, regexp, lit(idx)) + return _invoke_function_over_columns("regexp_extract_all", col, regexp, lit(idx)) @_try_remote_functions def regexp_replace( - string: "ColumnOrName", pattern: Union[str, Column], replacement: Union[str, Column] + col: "ColumnOrName", pattern: Union[str, Column], replacement: Union[str, Column] ) -> Column: r"""Replace all substrings of the specified string value that match regexp with replacement. @@ -15449,7 +15449,7 @@ def regexp_replace( Parameters ---------- - string : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str column name or column containing the string value pattern : :class:`~pyspark.sql.Column` or str column object or str containing the regexp pattern @@ -15469,19 +15469,19 @@ def regexp_replace( >>> df.select(regexp_replace("str", col("pattern"), col("replacement")).alias('d')).collect() [Row(d='-----')] """ - return _invoke_function_over_columns("regexp_replace", string, lit(pattern), lit(replacement)) + return _invoke_function_over_columns("regexp_replace", col, lit(pattern), lit(replacement)) @_try_remote_functions -def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns the substring that matches the Java regex `regexp` within the string `str`. +def regexp_substr(col: "ColumnOrName", regexp: "ColumnOrName") -> Column: + r"""Returns the substring that matches the Java regex `regexp` within the string `col`. If the regular expression is not found, the result is null. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15489,7 +15489,7 @@ def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Returns ------- :class:`~pyspark.sql.Column` - the substring that matches a Java regex within the string `str`. + the substring that matches a Java regex within the string `col`. Examples -------- @@ -15501,21 +15501,21 @@ def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: >>> df.select(regexp_substr("str", col("regexp")).alias('d')).collect() [Row(d='1')] """ - return _invoke_function_over_columns("regexp_substr", str, regexp) + return _invoke_function_over_columns("regexp_substr", col, regexp) @_try_remote_functions def regexp_instr( - str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None + col: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: - r"""Extract all strings in the `str` that match the Java regex `regexp` + r"""Extract all strings in the `col` that match the Java regex `regexp` and corresponding to the regex group index. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str target column to work on. regexp : :class:`~pyspark.sql.Column` or str regex pattern to apply. @@ -15525,7 +15525,7 @@ def regexp_instr( Returns ------- :class:`~pyspark.sql.Column` - all strings in the `str` that match a Java regex and corresponding to the regex group index. + all strings in the `col` that match a Java regex and corresponding to the regex group index. Examples -------- @@ -15540,9 +15540,9 @@ def regexp_instr( [Row(d=1)] """ if idx is None: - return _invoke_function_over_columns("regexp_instr", str, regexp) + return _invoke_function_over_columns("regexp_instr", col, regexp) else: - return _invoke_function_over_columns("regexp_instr", str, regexp, lit(idx)) + return _invoke_function_over_columns("regexp_instr", col, regexp, lit(idx)) @_try_remote_functions @@ -15733,8 +15733,8 @@ def unhex(col: "ColumnOrName") -> Column: @_try_remote_functions def uniform( - min: Union[Column, int, float], - max: Union[Column, int, float], + minimum: Union[Column, int, float], + maximum: Union[Column, int, float], seed: Optional[Union[Column, int]] = None, ) -> Column: """Returns a random value with independent and identically distributed (i.i.d.) values with the @@ -15747,9 +15747,9 @@ def uniform( Parameters ---------- - min : :class:`~pyspark.sql.Column`, int, or float + minimum : :class:`~pyspark.sql.Column`, int, or float Minimum value in the range. - max : :class:`~pyspark.sql.Column`, int, or float + maximum : :class:`~pyspark.sql.Column`, int, or float Maximum value in the range. seed : :class:`~pyspark.sql.Column` or int Optional random number seed to use. @@ -15778,16 +15778,16 @@ def uniform( | 83| +------------------+ """ - min = _enum_to_value(min) - min = lit(min) - max = _enum_to_value(max) - max = lit(max) + minimum = _enum_to_value(minimum) + minimum = lit(minimum) + maximum = _enum_to_value(maximum) + maximum = lit(maximum) if seed is None: - return _invoke_function_over_columns("uniform", min, max) + return _invoke_function_over_columns("uniform", minimum, maximum) else: seed = _enum_to_value(seed) seed = lit(seed) - return _invoke_function_over_columns("uniform", min, max, seed) + return _invoke_function_over_columns("uniform", minimum, maximum, seed) @_try_remote_functions @@ -15920,12 +15920,12 @@ def translate(srcCol: "ColumnOrName", matching: str, replace: str) -> Column: @_try_remote_functions -def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: +def to_binary(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: """ - Converts the input `col` to a binary value based on the supplied `format`. - The `format` can be a case-insensitive string literal of "hex", "utf-8", "utf8", + Converts the input `col` to a binary value based on the supplied `fmt`. + The `fmt` can be a case-insensitive string literal of "hex", "utf-8", "utf8", or "base64". By default, the binary format for conversion is "hex" if - `format` is omitted. The function returns NULL if at least one of the + `fmt` is omitted. The function returns NULL if at least one of the input parameters is NULL. .. versionadded:: 3.5.0 @@ -15934,7 +15934,7 @@ def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> C ---------- col : :class:`~pyspark.sql.Column` or str Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + fmt : :class:`~pyspark.sql.Column` or str, optional format to use to convert binary values. Examples @@ -15953,16 +15953,16 @@ def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> C >>> df.select(sf.try_to_binary(df.e).alias('r')).collect() [Row(r=bytearray(b'ABC'))] """ - if format is not None: - return _invoke_function_over_columns("to_binary", col, format) + if fmt is not None: + return _invoke_function_over_columns("to_binary", col, fmt) else: return _invoke_function_over_columns("to_binary", col) @_try_remote_functions -def to_char(col: "ColumnOrName", format: "ColumnOrName") -> Column: +def to_char(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: """ - Convert `col` to a string based on the `format`. + Convert `col` to a string based on the `fmt`. Throws an exception if the conversion fails. The format can consist of the following characters, case insensitive: '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the @@ -15980,7 +15980,7 @@ def to_char(col: "ColumnOrName", format: "ColumnOrName") -> Column: values but 'MI' prints a space. 'PR': Only allowed at the end of the format string; specifies that the result string will be wrapped by angle brackets if the input value is negative. - If `col` is a datetime, `format` shall be a valid datetime pattern, see + If `col` is a datetime, `fmt` shall be a valid datetime pattern, see Patterns. If `col` is a binary, it is converted to a string in one of the formats: 'base64': a base 64 string. @@ -15993,7 +15993,7 @@ def to_char(col: "ColumnOrName", format: "ColumnOrName") -> Column: ---------- col : :class:`~pyspark.sql.Column` or str Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + fmt : :class:`~pyspark.sql.Column` or str, optional format to use to convert char values. Examples @@ -16002,13 +16002,13 @@ def to_char(col: "ColumnOrName", format: "ColumnOrName") -> Column: >>> df.select(to_char(df.e, lit("$99.99")).alias('r')).collect() [Row(r='$78.12')] """ - return _invoke_function_over_columns("to_char", col, format) + return _invoke_function_over_columns("to_char", col, fmt) @_try_remote_functions -def to_varchar(col: "ColumnOrName", format: "ColumnOrName") -> Column: +def to_varchar(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: """ - Convert `col` to a string based on the `format`. + Convert `col` to a string based on the `fmt`. Throws an exception if the conversion fails. The format can consist of the following characters, case insensitive: '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the @@ -16026,7 +16026,7 @@ def to_varchar(col: "ColumnOrName", format: "ColumnOrName") -> Column: values but 'MI' prints a space. 'PR': Only allowed at the end of the format string; specifies that the result string will be wrapped by angle brackets if the input value is negative. - If `col` is a datetime, `format` shall be a valid datetime pattern, see + If `col` is a datetime, `fmt` shall be a valid datetime pattern, see Patterns. If `col` is a binary, it is converted to a string in one of the formats: 'base64': a base 64 string. @@ -16048,13 +16048,13 @@ def to_varchar(col: "ColumnOrName", format: "ColumnOrName") -> Column: >>> df.select(to_varchar(df.e, lit("$99.99")).alias('r')).collect() [Row(r='$78.12')] """ - return _invoke_function_over_columns("to_varchar", col, format) + return _invoke_function_over_columns("to_varchar", col, fmt) @_try_remote_functions -def to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: +def to_number(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: """ - Convert string 'col' to a number based on the string format 'format'. + Convert string 'col' to a number based on the string format 'fmt'. Throws an exception if the conversion fails. The format can consist of the following characters, case insensitive: '0' or '9': Specifies an expected digit between 0 and 9. A sequence of 0 or 9 in the @@ -16089,7 +16089,7 @@ def to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: >>> df.select(to_number(df.e, lit("$99.99")).alias('r')).collect() [Row(r=Decimal('78.12'))] """ - return _invoke_function_over_columns("to_number", col, format) + return _invoke_function_over_columns("to_number", col, fmt) @_try_remote_functions @@ -16106,10 +16106,10 @@ def replace( src : :class:`~pyspark.sql.Column` or str A column of string to be replaced. search : :class:`~pyspark.sql.Column` or str - A column of string, If `search` is not found in `str`, `str` is returned unchanged. + A column of string, If `search` is not found in `col`, `col` is returned unchanged. replace : :class:`~pyspark.sql.Column` or str, optional A column of string, If `replace` is not specified or is an empty string, - nothing replaces the string that is removed from `str`. + nothing replaces the string that is removed from `col`. Examples -------- @@ -16129,11 +16129,11 @@ def replace( @_try_remote_functions def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnOrName") -> Column: """ - Splits `str` by delimiter and return requested part of the split (1-based). + Splits `col` by delimiter and return requested part of the split (1-based). If any input is null, returns null. if `partNum` is out of range of split parts, returns empty string. If `partNum` is 0, throws an error. If `partNum` is negative, the parts are counted backward from the end of the string. - If the `delimiter` is an empty string, the `str` is not split. + If the `delimiter` is an empty string, the `col` is not split. .. versionadded:: 3.5.0 @@ -16174,22 +16174,22 @@ def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnO @_try_remote_functions def substr( - str: "ColumnOrName", pos: "ColumnOrName", len: Optional["ColumnOrName"] = None + col: "ColumnOrName", pos: "ColumnOrName", length: Optional["ColumnOrName"] = None ) -> Column: """ - Returns the substring of `str` that starts at `pos` and is of length `len`, - or the slice of byte array that starts at `pos` and is of length `len`. + Returns the substring of `col` that starts at `pos` and is of length `length`, + or the slice of byte array that starts at `pos` and is of length `length`. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or column name + col : :class:`~pyspark.sql.Column` or column name A column of string. pos : :class:`~pyspark.sql.Column` or column name - A column of string, the substring of `str` that starts at `pos`. - len : :class:`~pyspark.sql.Column` or column name, optional - A column of string, the substring of `str` is of length `len`. + A column of string, the substring of `col` that starts at `pos`. + length : :class:`~pyspark.sql.Column` or column name, optional + A column of string, the substring of `col` is of length `length`. Returns ------- @@ -16221,10 +16221,10 @@ def substr( |Spark SQL| 5| 1| k SQL| +---------+---+---+------------------------+ """ - if len is not None: - return _invoke_function_over_columns("substr", str, pos, len) + if length is not None: + return _invoke_function_over_columns("substr", col, pos, length) else: - return _invoke_function_over_columns("substr", str, pos) + return _invoke_function_over_columns("substr", col, pos) @_try_remote_functions @@ -16446,7 +16446,7 @@ def parse_url( @_try_remote_functions -def printf(format: "ColumnOrName", *cols: "ColumnOrName") -> Column: +def printf(fmt: "ColumnOrName", *cols: "ColumnOrName") -> Column: """ Formats the arguments in printf-style and returns the result as a string column. @@ -16454,7 +16454,7 @@ def printf(format: "ColumnOrName", *cols: "ColumnOrName") -> Column: Parameters ---------- - format : :class:`~pyspark.sql.Column` or str + fmt : :class:`~pyspark.sql.Column` or str string that can contain embedded format tags and used as result column's value cols : :class:`~pyspark.sql.Column` or str column names or :class:`~pyspark.sql.Column`\\s to be used in formatting @@ -16474,11 +16474,11 @@ def printf(format: "ColumnOrName", *cols: "ColumnOrName") -> Column: from pyspark.sql.classic.column import _to_seq, _to_java_column sc = _get_active_spark_context() - return _invoke_function("printf", _to_java_column(format), _to_seq(sc, cols, _to_java_column)) + return _invoke_function("printf", _to_java_column(fmt), _to_seq(sc, cols, _to_java_column)) @_try_remote_functions -def url_decode(str: "ColumnOrName") -> Column: +def url_decode(col: "ColumnOrName") -> Column: """ URL function: Decodes a URL-encoded string in 'application/x-www-form-urlencoded' format to its original format. @@ -16487,7 +16487,7 @@ def url_decode(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of strings, each representing a URL-encoded string. Returns @@ -16552,11 +16552,11 @@ def url_decode(str: "ColumnOrName") -> Column: | ~!@#$%^&*()_+| +---------------+ """ - return _invoke_function_over_columns("url_decode", str) + return _invoke_function_over_columns("url_decode", col) @_try_remote_functions -def try_url_decode(str: "ColumnOrName") -> Column: +def try_url_decode(col: "ColumnOrName") -> Column: """ This is a special version of `url_decode` that performs the same operation, but returns a NULL value instead of raising an error if the decoding cannot be performed. @@ -16565,7 +16565,7 @@ def try_url_decode(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of strings, each representing a URL-encoded string. Returns @@ -16597,11 +16597,11 @@ def try_url_decode(str: "ColumnOrName") -> Column: | NULL| +-------------------+ """ - return _invoke_function_over_columns("try_url_decode", str) + return _invoke_function_over_columns("try_url_decode", col) @_try_remote_functions -def url_encode(str: "ColumnOrName") -> Column: +def url_encode(col: "ColumnOrName") -> Column: """ URL function: Encodes a string into a URL-encoded string in 'application/x-www-form-urlencoded' format. @@ -16610,7 +16610,7 @@ def url_encode(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of strings, each representing a string to be URL-encoded. Returns @@ -16675,15 +16675,15 @@ def url_encode(str: "ColumnOrName") -> Column: |%7E%21%40%23%24%25%5E%26*%28%29_%2B| +-----------------------------------+ """ - return _invoke_function_over_columns("url_encode", str) + return _invoke_function_over_columns("url_encode", col) @_try_remote_functions def position( - substr: "ColumnOrName", str: "ColumnOrName", start: Optional["ColumnOrName"] = None + substr: "ColumnOrName", col: "ColumnOrName", start: Optional["ColumnOrName"] = None ) -> Column: """ - Returns the position of the first occurrence of `substr` in `str` after position `start`. + Returns the position of the first occurrence of `substr` in `col` after position `start`. The given `start` and return value are 1-based. .. versionadded:: 3.5.0 @@ -16692,7 +16692,7 @@ def position( ---------- substr : :class:`~pyspark.sql.Column` or str A column of string, substring. - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of string. start : :class:`~pyspark.sql.Column` or str, optional A column of string, start position. @@ -16719,13 +16719,13 @@ def position( +-----------------+ """ if start is not None: - return _invoke_function_over_columns("position", substr, str, start) + return _invoke_function_over_columns("position", substr, col, start) else: - return _invoke_function_over_columns("position", substr, str) + return _invoke_function_over_columns("position", substr, col) @_try_remote_functions -def endswith(str: "ColumnOrName", suffix: "ColumnOrName") -> Column: +def endswith(col: "ColumnOrName", suffix: "ColumnOrName") -> Column: """ Returns a boolean. The value is True if str ends with suffix. Returns NULL if either input expression is NULL. Otherwise, returns False. @@ -16735,7 +16735,7 @@ def endswith(str: "ColumnOrName", suffix: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of string. suffix : :class:`~pyspark.sql.Column` or str A column of string, the suffix. @@ -16759,11 +16759,11 @@ def endswith(str: "ColumnOrName", suffix: "ColumnOrName") -> Column: | true| false| +--------------+--------------+ """ - return _invoke_function_over_columns("endswith", str, suffix) + return _invoke_function_over_columns("endswith", col, suffix) @_try_remote_functions -def startswith(str: "ColumnOrName", prefix: "ColumnOrName") -> Column: +def startswith(col: "ColumnOrName", prefix: "ColumnOrName") -> Column: """ Returns a boolean. The value is True if str starts with prefix. Returns NULL if either input expression is NULL. Otherwise, returns False. @@ -16773,7 +16773,7 @@ def startswith(str: "ColumnOrName", prefix: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A column of string. prefix : :class:`~pyspark.sql.Column` or str A column of string, the prefix. @@ -16797,7 +16797,7 @@ def startswith(str: "ColumnOrName", prefix: "ColumnOrName") -> Column: | true| false| +----------------+----------------+ """ - return _invoke_function_over_columns("startswith", str, prefix) + return _invoke_function_over_columns("startswith", col, prefix) @_try_remote_functions @@ -16827,15 +16827,15 @@ def char(col: "ColumnOrName") -> Column: @_try_remote_functions -def btrim(str: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: +def btrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: """ - Remove the leading and trailing `trim` characters from `str`. + Remove the leading and trailing `trim` characters from `col`. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. trim : :class:`~pyspark.sql.Column` or str, optional The trim string characters to trim, the default value is a single space @@ -16851,13 +16851,13 @@ def btrim(str: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: [Row(r='SparkSQL')] """ if trim is not None: - return _invoke_function_over_columns("btrim", str, trim) + return _invoke_function_over_columns("btrim", col, trim) else: - return _invoke_function_over_columns("btrim", str) + return _invoke_function_over_columns("btrim", col) @_try_remote_functions -def char_length(str: "ColumnOrName") -> Column: +def char_length(col: "ColumnOrName") -> Column: """ Returns the character length of string data or number of bytes of binary data. The length of string data includes the trailing spaces. @@ -16867,7 +16867,7 @@ def char_length(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. Examples @@ -16880,11 +16880,11 @@ def char_length(str: "ColumnOrName") -> Column: | 8| +---------------------+ """ - return _invoke_function_over_columns("char_length", str) + return _invoke_function_over_columns("char_length", col) @_try_remote_functions -def character_length(str: "ColumnOrName") -> Column: +def character_length(col: "ColumnOrName") -> Column: """ Returns the character length of string data or number of bytes of binary data. The length of string data includes the trailing spaces. @@ -16894,7 +16894,7 @@ def character_length(str: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. Examples @@ -16907,11 +16907,11 @@ def character_length(str: "ColumnOrName") -> Column: | 8| +--------------------------+ """ - return _invoke_function_over_columns("character_length", str) + return _invoke_function_over_columns("character_length", col) @_try_remote_functions -def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: +def try_to_binary(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: """ This is a special version of `to_binary` that performs the same operation, but returns a NULL value instead of raising an error if the conversion cannot be performed. @@ -16922,7 +16922,7 @@ def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) ---------- col : :class:`~pyspark.sql.Column` or str Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + fmt : :class:`~pyspark.sql.Column` or str, optional format to use to convert binary values. Examples @@ -16957,16 +16957,16 @@ def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) | NULL| +-----------------------------+ """ - if format is not None: - return _invoke_function_over_columns("try_to_binary", col, format) + if fmt is not None: + return _invoke_function_over_columns("try_to_binary", col, fmt) else: return _invoke_function_over_columns("try_to_binary", col) @_try_remote_functions -def try_to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: +def try_to_number(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: """ - Convert string 'col' to a number based on the string format `format`. Returns NULL if the + Convert string 'col' to a number based on the string format `fmt`. Returns NULL if the string 'col' does not match the expected format. The format follows the same semantics as the to_number function. @@ -16976,7 +16976,7 @@ def try_to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: ---------- col : :class:`~pyspark.sql.Column` or str Input column or strings. - format : :class:`~pyspark.sql.Column` or str, optional + fmt : :class:`~pyspark.sql.Column` or str, optional format to use to convert number values. Examples @@ -17008,7 +17008,7 @@ def try_to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: |NULL| +----+ """ - return _invoke_function_over_columns("try_to_number", col, format) + return _invoke_function_over_columns("try_to_number", col, fmt) @_try_remote_functions @@ -17077,17 +17077,17 @@ def elt(*inputs: "ColumnOrName") -> Column: @_try_remote_functions -def find_in_set(str: "ColumnOrName", str_array: "ColumnOrName") -> Column: +def find_in_set(col: "ColumnOrName", str_array: "ColumnOrName") -> Column: """ - Returns the index (1-based) of the given string (`str`) in the comma-delimited - list (`strArray`). Returns 0, if the string was not found or if the given string (`str`) + Returns the index (1-based) of the given string (`col`) in the comma-delimited + list (`strArray`). Returns 0, if the string was not found or if the given string (`col`) contains a comma. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str The given string to be found. str_array : :class:`~pyspark.sql.Column` or str The comma-delimited list. @@ -17098,12 +17098,12 @@ def find_in_set(str: "ColumnOrName", str_array: "ColumnOrName") -> Column: >>> df.select(find_in_set(df.a, df.b).alias('r')).collect() [Row(r=3)] """ - return _invoke_function_over_columns("find_in_set", str, str_array) + return _invoke_function_over_columns("find_in_set", col, str_array) @_try_remote_functions def like( - str: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None + col: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None ) -> Column: """ Returns true if str matches `pattern` with `escape`, @@ -17114,7 +17114,7 @@ def like( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A string. pattern : :class:`~pyspark.sql.Column` or str A string. The pattern is a string which is matched literally, with @@ -17146,14 +17146,14 @@ def like( [Row(r=True)] """ if escapeChar is not None: - return _invoke_function_over_columns("like", str, pattern, escapeChar) + return _invoke_function_over_columns("like", col, pattern, escapeChar) else: - return _invoke_function_over_columns("like", str, pattern) + return _invoke_function_over_columns("like", col, pattern) @_try_remote_functions def ilike( - str: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None + col: "ColumnOrName", pattern: "ColumnOrName", escapeChar: Optional["Column"] = None ) -> Column: """ Returns true if str matches `pattern` with `escape` case-insensitively, @@ -17164,7 +17164,7 @@ def ilike( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str A string. pattern : :class:`~pyspark.sql.Column` or str A string. The pattern is a string which is matched literally, with @@ -17196,21 +17196,21 @@ def ilike( [Row(r=True)] """ if escapeChar is not None: - return _invoke_function_over_columns("ilike", str, pattern, escapeChar) + return _invoke_function_over_columns("ilike", col, pattern, escapeChar) else: - return _invoke_function_over_columns("ilike", str, pattern) + return _invoke_function_over_columns("ilike", col, pattern) @_try_remote_functions -def lcase(str: "ColumnOrName") -> Column: +def lcase(col: "ColumnOrName") -> Column: """ - Returns `str` with all characters changed to lowercase. + Returns `col` with all characters changed to lowercase. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. Examples @@ -17227,15 +17227,15 @@ def lcase(str: "ColumnOrName") -> Column: @_try_remote_functions -def ucase(str: "ColumnOrName") -> Column: +def ucase(col: "ColumnOrName") -> Column: """ - Returns `str` with all characters changed to uppercase. + Returns `col` with all characters changed to uppercase. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. Examples @@ -17248,23 +17248,23 @@ def ucase(str: "ColumnOrName") -> Column: | SPARK| +------------+ """ - return _invoke_function_over_columns("ucase", str) + return _invoke_function_over_columns("ucase", col) @_try_remote_functions -def left(str: "ColumnOrName", len: "ColumnOrName") -> Column: +def left(col: "ColumnOrName", length: "ColumnOrName") -> Column: """ - Returns the leftmost `len`(`len` can be string type) characters from the string `str`, - if `len` is less or equal than 0 the result is an empty string. + Returns the leftmost `length`(`length` can be string type) characters from the string `col`, + if `length` is less or equal than 0 the result is an empty string. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + col : :class:`~pyspark.sql.Column` or str Input column or strings. - len : :class:`~pyspark.sql.Column` or str - Input column or strings, the leftmost `len`. + length : :class:`~pyspark.sql.Column` or str + Input column or strings, the leftmost `length`. Examples -------- @@ -17272,14 +17272,14 @@ def left(str: "ColumnOrName", len: "ColumnOrName") -> Column: >>> df.select(left(df.a, df.b).alias('r')).collect() [Row(r='Spa')] """ - return _invoke_function_over_columns("left", str, len) + return _invoke_function_over_columns("left", col, length) @_try_remote_functions -def right(str: "ColumnOrName", len: "ColumnOrName") -> Column: +def right(col: "ColumnOrName", length: "ColumnOrName") -> Column: """ - Returns the rightmost `len`(`len` can be string type) characters from the string `str`, - if `len` is less or equal than 0 the result is an empty string. + Returns the rightmost `length`(`length` can be string type) characters from the string `col`, + if `length` is less or equal than 0 the result is an empty string. .. versionadded:: 3.5.0 @@ -17288,7 +17288,7 @@ def right(str: "ColumnOrName", len: "ColumnOrName") -> Column: str : :class:`~pyspark.sql.Column` or str Input column or strings. len : :class:`~pyspark.sql.Column` or str - Input column or strings, the rightmost `len`. + Input column or strings, the rightmost `length`. Examples -------- @@ -17296,7 +17296,7 @@ def right(str: "ColumnOrName", len: "ColumnOrName") -> Column: >>> df.select(right(df.a, df.b).alias('r')).collect() [Row(r='SQL')] """ - return _invoke_function_over_columns("right", str, len) + return _invoke_function_over_columns("right", col, length) @_try_remote_functions diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 39db72b235bf9..3d5f34537f3f8 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -808,7 +808,7 @@ def test_overlay(self): self.check_error( exception=pe.exception, errorClass="NOT_COLUMN_OR_INT_OR_STR", - messageParameters={"arg_name": "len", "arg_type": "float"}, + messageParameters={"arg_name": "length", "arg_type": "float"}, ) def test_percentile(self): From 75b8757bd74897bfc1cdc74c9dabd8db3135e59a Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Fri, 27 Dec 2024 23:58:19 +0100 Subject: [PATCH 2/8] correct call --- python/pyspark/sql/connect/functions/builtin.py | 6 +++--- python/pyspark/sql/functions/builtin.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index f64884e15e6ed..f855c12d1f0da 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -2725,9 +2725,9 @@ def regexp_extract_all( col: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: if idx is None: - return _invoke_function_over_columns("regexp_extract_all", str, regexp) + return _invoke_function_over_columns("regexp_extract_all", col, regexp) else: - return _invoke_function_over_columns("regexp_extract_all", str, regexp, lit(idx)) + return _invoke_function_over_columns("regexp_extract_all", col, regexp, lit(idx)) regexp_extract_all.__doc__ = pysparkfuncs.regexp_extract_all.__doc__ @@ -2856,7 +2856,7 @@ def split_part(src: "ColumnOrName", delimiter: "ColumnOrName", partNum: "ColumnO def substr( col: "ColumnOrName", pos: "ColumnOrName", length: Optional["ColumnOrName"] = None ) -> Column: - if len is not None: + if length is not None: return _invoke_function_over_columns("substr", col, pos, length) else: return _invoke_function_over_columns("substr", col, pos) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index a2f6ede6d7142..83f369bbd8bd8 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -17223,7 +17223,7 @@ def lcase(col: "ColumnOrName") -> Column: | spark| +------------+ """ - return _invoke_function_over_columns("lcase", str) + return _invoke_function_over_columns("lcase", col) @_try_remote_functions From 071976d3db0825a5f4e3f0b1e54e198cf4c7f334 Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 00:43:49 +0100 Subject: [PATCH 3/8] correct test --- python/pyspark/sql/tests/connect/test_connect_function.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_function.py b/python/pyspark/sql/tests/connect/test_connect_function.py index d1e2558305291..7baba2ca1537b 100644 --- a/python/pyspark/sql/tests/connect/test_connect_function.py +++ b/python/pyspark/sql/tests/connect/test_connect_function.py @@ -2235,14 +2235,14 @@ def test_date_ts_functions(self): (CF.to_date, SF.to_date), ]: self.assert_eq( - cdf.select(cfunc(cdf.ts1, format="yyyy-MM-dd HH:mm:ss")).toPandas(), - sdf.select(sfunc(sdf.ts1, format="yyyy-MM-dd HH:mm:ss")).toPandas(), + cdf.select(cfunc(cdf.ts1, fmt="yyyy-MM-dd HH:mm:ss")).toPandas(), + sdf.select(sfunc(sdf.ts1, fmt="yyyy-MM-dd HH:mm:ss")).toPandas(), ) self.compare_by_show( # [left]: datetime64[ns, America/Los_Angeles] # [right]: datetime64[ns] - cdf.select(CF.to_timestamp(cdf.ts1, format="yyyy-MM-dd HH:mm:ss")), - sdf.select(SF.to_timestamp(sdf.ts1, format="yyyy-MM-dd HH:mm:ss")), + cdf.select(CF.to_timestamp(cdf.ts1, fmt="yyyy-MM-dd HH:mm:ss")), + sdf.select(SF.to_timestamp(sdf.ts1, fmt="yyyy-MM-dd HH:mm:ss")), ) # With tz parameter From d3eedcc1be98e689daad10b4706124a17e935cc3 Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 00:48:10 +0100 Subject: [PATCH 4/8] format in spark connect --- .../pyspark/sql/connect/functions/builtin.py | 92 +++++++++---------- 1 file changed, 46 insertions(+), 46 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index f855c12d1f0da..0d4c1a6016ae1 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -2803,9 +2803,9 @@ def translate(srcCol: "ColumnOrName", matching: str, replace: str) -> Column: translate.__doc__ = pysparkfuncs.translate.__doc__ -def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: - if format is not None: - return _invoke_function_over_columns("to_binary", col, format) +def to_binary(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: + if fmt is not None: + return _invoke_function_over_columns("to_binary", col, fmt) else: return _invoke_function_over_columns("to_binary", col) @@ -2813,22 +2813,22 @@ def to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> C to_binary.__doc__ = pysparkfuncs.to_binary.__doc__ -def to_char(col: "ColumnOrName", format: "ColumnOrName") -> Column: - return _invoke_function_over_columns("to_char", col, format) +def to_char(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: + return _invoke_function_over_columns("to_char", col, fmt) to_char.__doc__ = pysparkfuncs.to_char.__doc__ -def to_varchar(col: "ColumnOrName", format: "ColumnOrName") -> Column: - return _invoke_function_over_columns("to_varchar", col, format) +def to_varchar(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: + return _invoke_function_over_columns("to_varchar", col, fmt) to_varchar.__doc__ = pysparkfuncs.to_varchar.__doc__ -def to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: - return _invoke_function_over_columns("to_number", col, format) +def to_number(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: + return _invoke_function_over_columns("to_number", col, fmt) to_number.__doc__ = pysparkfuncs.to_number.__doc__ @@ -2889,8 +2889,8 @@ def try_parse_url( try_parse_url.__doc__ = pysparkfuncs.try_parse_url.__doc__ -def printf(format: "ColumnOrName", *cols: "ColumnOrName") -> Column: - return _invoke_function("printf", _to_col(format), *[_to_col(c) for c in cols]) +def printf(fmt: "ColumnOrName", *cols: "ColumnOrName") -> Column: + return _invoke_function("printf", _to_col(fmt), *[_to_col(c) for c in cols]) printf.__doc__ = pysparkfuncs.printf.__doc__ @@ -2950,9 +2950,9 @@ def char(col: "ColumnOrName") -> Column: char.__doc__ = pysparkfuncs.char.__doc__ -def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: - if format is not None: - return _invoke_function_over_columns("try_to_binary", col, format) +def try_to_binary(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: + if fmt is not None: + return _invoke_function_over_columns("try_to_binary", col, fmt) else: return _invoke_function_over_columns("try_to_binary", col) @@ -2960,8 +2960,8 @@ def try_to_binary(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) try_to_binary.__doc__ = pysparkfuncs.try_to_binary.__doc__ -def try_to_number(col: "ColumnOrName", format: "ColumnOrName") -> Column: - return _invoke_function_over_columns("try_to_number", col, format) +def try_to_number(col: "ColumnOrName", fmt: "ColumnOrName") -> Column: + return _invoke_function_over_columns("try_to_number", col, fmt) try_to_number.__doc__ = pysparkfuncs.try_to_number.__doc__ @@ -3143,8 +3143,8 @@ def localtimestamp() -> Column: localtimestamp.__doc__ = pysparkfuncs.localtimestamp.__doc__ -def date_format(date: "ColumnOrName", format: str) -> Column: - return _invoke_function("date_format", _to_col(date), lit(format)) +def date_format(date: "ColumnOrName", fmt: str) -> Column: + return _invoke_function("date_format", _to_col(date), lit(fmt)) date_format.__doc__ = pysparkfuncs.date_format.__doc__ @@ -3340,11 +3340,11 @@ def months_between(date1: "ColumnOrName", date2: "ColumnOrName", roundOff: bool months_between.__doc__ = pysparkfuncs.months_between.__doc__ -def to_date(col: "ColumnOrName", format: Optional[str] = None) -> Column: - if format is None: +def to_date(col: "ColumnOrName", fmt: Optional[str] = None) -> Column: + if fmt is None: return _invoke_function_over_columns("to_date", col) else: - return _invoke_function("to_date", _to_col(col), lit(format)) + return _invoke_function("to_date", _to_col(col), lit(fmt)) to_date.__doc__ = pysparkfuncs.to_date.__doc__ @@ -3384,23 +3384,23 @@ def to_timestamp(col: "ColumnOrName") -> Column: @overload -def to_timestamp(col: "ColumnOrName", format: str) -> Column: +def to_timestamp(col: "ColumnOrName", fmt: str) -> Column: ... -def to_timestamp(col: "ColumnOrName", format: Optional[str] = None) -> Column: - if format is None: +def to_timestamp(col: "ColumnOrName", fmt: Optional[str] = None) -> Column: + if fmt is None: return _invoke_function_over_columns("to_timestamp", col) else: - return _invoke_function("to_timestamp", _to_col(col), lit(format)) + return _invoke_function("to_timestamp", _to_col(col), lit(fmt)) to_timestamp.__doc__ = pysparkfuncs.to_timestamp.__doc__ -def try_to_timestamp(col: "ColumnOrName", format: Optional["ColumnOrName"] = None) -> Column: - if format is not None: - return _invoke_function_over_columns("try_to_timestamp", col, format) +def try_to_timestamp(col: "ColumnOrName", fmt: Optional["ColumnOrName"] = None) -> Column: + if fmt is not None: + return _invoke_function_over_columns("try_to_timestamp", col, fmt) else: return _invoke_function_over_columns("try_to_timestamp", col) @@ -3471,15 +3471,15 @@ def xpath_string(xml: "ColumnOrName", path: "ColumnOrName") -> Column: xpath_string.__doc__ = pysparkfuncs.xpath_string.__doc__ -def trunc(date: "ColumnOrName", format: str) -> Column: - return _invoke_function("trunc", _to_col(date), lit(format)) +def trunc(date: "ColumnOrName", fmt: str) -> Column: + return _invoke_function("trunc", _to_col(date), lit(fmt)) trunc.__doc__ = pysparkfuncs.trunc.__doc__ -def date_trunc(format: str, timestamp: "ColumnOrName") -> Column: - return _invoke_function("date_trunc", lit(format), _to_col(timestamp)) +def date_trunc(fmt: str, timestamp: "ColumnOrName") -> Column: + return _invoke_function("date_trunc", lit(fmt), _to_col(timestamp)) date_trunc.__doc__ = pysparkfuncs.date_trunc.__doc__ @@ -3499,15 +3499,15 @@ def last_day(date: "ColumnOrName") -> Column: last_day.__doc__ = pysparkfuncs.last_day.__doc__ -def from_unixtime(timestamp: "ColumnOrName", format: str = "yyyy-MM-dd HH:mm:ss") -> Column: - return _invoke_function("from_unixtime", _to_col(timestamp), lit(format)) +def from_unixtime(timestamp: "ColumnOrName", fmt: str = "yyyy-MM-dd HH:mm:ss") -> Column: + return _invoke_function("from_unixtime", _to_col(timestamp), lit(fmt)) from_unixtime.__doc__ = pysparkfuncs.from_unixtime.__doc__ @overload -def unix_timestamp(timestamp: "ColumnOrName", format: str = ...) -> Column: +def unix_timestamp(timestamp: "ColumnOrName", fmt: str = ...) -> Column: ... @@ -3517,11 +3517,11 @@ def unix_timestamp() -> Column: def unix_timestamp( - timestamp: Optional["ColumnOrName"] = None, format: str = "yyyy-MM-dd HH:mm:ss" + timestamp: Optional["ColumnOrName"] = None, fmt: str = "yyyy-MM-dd HH:mm:ss" ) -> Column: if timestamp is None: return _invoke_function("unix_timestamp") - return _invoke_function("unix_timestamp", _to_col(timestamp), lit(format)) + return _invoke_function("unix_timestamp", _to_col(timestamp), lit(fmt)) unix_timestamp.__doc__ = pysparkfuncs.unix_timestamp.__doc__ @@ -3653,10 +3653,10 @@ def session_window(timeColumn: "ColumnOrName", gapDuration: Union[Column, str]) def to_unix_timestamp( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: - if format is not None: - return _invoke_function_over_columns("to_unix_timestamp", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_unix_timestamp", timestamp, fmt) else: return _invoke_function_over_columns("to_unix_timestamp", timestamp) @@ -3666,10 +3666,10 @@ def to_unix_timestamp( def to_timestamp_ltz( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: - if format is not None: - return _invoke_function_over_columns("to_timestamp_ltz", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_timestamp_ltz", timestamp, fmt) else: return _invoke_function_over_columns("to_timestamp_ltz", timestamp) @@ -3679,10 +3679,10 @@ def to_timestamp_ltz( def to_timestamp_ntz( timestamp: "ColumnOrName", - format: Optional["ColumnOrName"] = None, + fmt: Optional["ColumnOrName"] = None, ) -> Column: - if format is not None: - return _invoke_function_over_columns("to_timestamp_ntz", timestamp, format) + if fmt is not None: + return _invoke_function_over_columns("to_timestamp_ntz", timestamp, fmt) else: return _invoke_function_over_columns("to_timestamp_ntz", timestamp) From a0643c20e4858f4c54836bbb50b570b5b7ee4bed Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 02:26:16 +0100 Subject: [PATCH 5/8] alignement --- python/pyspark/sql/connect/functions/builtin.py | 4 ++-- python/pyspark/sql/functions/builtin.py | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 0d4c1a6016ae1..d6c7dc593ee96 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -2734,9 +2734,9 @@ def regexp_extract_all( def regexp_replace( - string: "ColumnOrName", pattern: Union[str, Column], replacement: Union[str, Column] + col: "ColumnOrName", pattern: Union[str, Column], replacement: Union[str, Column] ) -> Column: - return _invoke_function_over_columns("regexp_replace", string, lit(pattern), lit(replacement)) + return _invoke_function_over_columns("regexp_replace", col, lit(pattern), lit(replacement)) regexp_replace.__doc__ = pysparkfuncs.regexp_replace.__doc__ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 83f369bbd8bd8..5edcc646f1d20 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6325,10 +6325,10 @@ def width_bucket( +----+-------+--------+---+------------------------------------+ | v|minimum| maximum| n|width_bucket(v, minimum, maximum, n)| +----+-------+--------+---+------------------------------------+ - | 5.3| 0.2|10.6 | 5| 3| - |-2.1| 1.3| 3.4 | 3| 0| - | 8.1| 0.0| 5.7 | 4| 5| - |-0.9| 5.2| 0.5 | 2| 3| + | 5.3| 0.2| 10.6| 5| 3| + |-2.1| 1.3| 3.4| 3| 0| + | 8.1| 0.0| 5.7| 4| 5| + |-0.9| 5.2| 0.5| 2| 3| +----+-------+--------+---+------------------------------------+ """ numBucket = _enum_to_value(numBucket) From 71b4059b8163a0c347169d01828e40b090a9d05e Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 03:37:59 +0100 Subject: [PATCH 6/8] add --- python/pyspark/sql/functions/builtin.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5edcc646f1d20..47289649fa35d 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6329,7 +6329,7 @@ def width_bucket( |-2.1| 1.3| 3.4| 3| 0| | 8.1| 0.0| 5.7| 4| 5| |-0.9| 5.2| 0.5| 2| 3| - +----+-------+--------+---+------------------------------------+ + +----+-------+--------+---+------------------------------------+ """ numBucket = _enum_to_value(numBucket) numBucket = lit(numBucket) if isinstance(numBucket, int) else numBucket From 73b75fbae511de360478c6dfcc1b4fa7f9b144ae Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 03:38:09 +0100 Subject: [PATCH 7/8] remove space --- python/pyspark/sql/functions/builtin.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 47289649fa35d..5edcc646f1d20 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6329,7 +6329,7 @@ def width_bucket( |-2.1| 1.3| 3.4| 3| 0| | 8.1| 0.0| 5.7| 4| 5| |-0.9| 5.2| 0.5| 2| 3| - +----+-------+--------+---+------------------------------------+ + +----+-------+--------+---+------------------------------------+ """ numBucket = _enum_to_value(numBucket) numBucket = lit(numBucket) if isinstance(numBucket, int) else numBucket From 9e6eb2972a74e4a52067f600d77b72ed946545d1 Mon Sep 17 00:00:00 2001 From: Skander Boudawara Date: Sat, 28 Dec 2024 03:38:54 +0100 Subject: [PATCH 8/8] close columns --- python/pyspark/sql/functions/builtin.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5edcc646f1d20..f7ac007efe44e 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6322,14 +6322,14 @@ def width_bucket( ... (-0.9, 5.2, 0.5, 2)], ... ['v', 'minimum', 'maximum', 'n']) >>> df.select("*", sf.width_bucket('v', 'minimum', 'maximum', 'n')).show() - +----+-------+--------+---+------------------------------------+ - | v|minimum| maximum| n|width_bucket(v, minimum, maximum, n)| - +----+-------+--------+---+------------------------------------+ - | 5.3| 0.2| 10.6| 5| 3| - |-2.1| 1.3| 3.4| 3| 0| - | 8.1| 0.0| 5.7| 4| 5| - |-0.9| 5.2| 0.5| 2| 3| - +----+-------+--------+---+------------------------------------+ + +----+-------+-------+---+------------------------------------+ + | v|minimum|maximum| n|width_bucket(v, minimum, maximum, n)| + +----+-------+-------+---+------------------------------------+ + | 5.3| 0.2| 10.6| 5| 3| + |-2.1| 1.3| 3.4| 3| 0| + | 8.1| 0.0| 5.7| 4| 5| + |-0.9| 5.2| 0.5| 2| 3| + +----+-------+-------+---+------------------------------------+ """ numBucket = _enum_to_value(numBucket) numBucket = lit(numBucket) if isinstance(numBucket, int) else numBucket