From 78135dc204e4fc18829a3698f1f253f90b7da301 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 8 Oct 2024 12:47:04 +0800 Subject: [PATCH] [SPARK-49889][PYTHON] Add argument `trim` for functions`trim/ltrim/rtrim` ### What changes were proposed in this pull request? Add argument `trim` for functions`trim/ltrim/rtrim` ### Why are the changes needed? this argument is missing in PySpark: we can specify the it in scala side but cannot do it in python. ### Does this PR introduce _any_ user-facing change? yes, new argument supported ### How was this patch tested? added doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #48363 from zhengruifeng/func_trim_str. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../pyspark/sql/connect/functions/builtin.py | 21 ++- python/pyspark/sql/functions/builtin.py | 131 +++++++++++++----- .../org/apache/spark/sql/functions.scala | 27 +++- 3 files changed, 139 insertions(+), 40 deletions(-) diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 27b12fff3c0ac..db12e085468a0 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -2394,22 +2394,31 @@ def unbase64(col: "ColumnOrName") -> Column: unbase64.__doc__ = pysparkfuncs.unbase64.__doc__ -def ltrim(col: "ColumnOrName") -> Column: - return _invoke_function_over_columns("ltrim", col) +def ltrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: + if trim is not None: + return _invoke_function_over_columns("ltrim", trim, col) + else: + return _invoke_function_over_columns("ltrim", col) ltrim.__doc__ = pysparkfuncs.ltrim.__doc__ -def rtrim(col: "ColumnOrName") -> Column: - return _invoke_function_over_columns("rtrim", col) +def rtrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: + if trim is not None: + return _invoke_function_over_columns("rtrim", trim, col) + else: + return _invoke_function_over_columns("rtrim", col) rtrim.__doc__ = pysparkfuncs.rtrim.__doc__ -def trim(col: "ColumnOrName") -> Column: - return _invoke_function_over_columns("trim", col) +def trim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: + if trim is not None: + return _invoke_function_over_columns("trim", trim, col) + else: + return _invoke_function_over_columns("trim", col) trim.__doc__ = pysparkfuncs.trim.__doc__ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 4ca39562cb20b..beed832e36067 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -10913,7 +10913,7 @@ def unbase64(col: "ColumnOrName") -> Column: @_try_remote_functions -def ltrim(col: "ColumnOrName") -> Column: +def ltrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: """ Trim the spaces from left end for the specified string value. @@ -10926,6 +10926,10 @@ def ltrim(col: "ColumnOrName") -> Column: ---------- col : :class:`~pyspark.sql.Column` or str target column to work on. + trim : :class:`~pyspark.sql.Column` or str, optional + The trim string characters to trim, the default value is a single space + + .. versionadded:: 4.0.0 Returns ------- @@ -10934,21 +10938,40 @@ def ltrim(col: "ColumnOrName") -> Column: Examples -------- + Example 1: Trim the spaces + + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([" Spark", "Spark ", " Spark"], "STRING") - >>> df.select(ltrim("value").alias("r")).withColumn("length", length("r")).show() - +-------+------+ - | r|length| - +-------+------+ - | Spark| 5| - |Spark | 7| - | Spark| 5| - +-------+------+ + >>> df.select("*", sf.ltrim("value")).show() + +--------+------------+ + | value|ltrim(value)| + +--------+------------+ + | Spark| Spark| + | Spark | Spark | + | Spark| Spark| + +--------+------------+ + + Example 2: Trim specified characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame(["***Spark", "Spark**", "*Spark"], "STRING") + >>> df.select("*", sf.ltrim("value", sf.lit("*"))).show() + +--------+--------------------------+ + | value|TRIM(LEADING * FROM value)| + +--------+--------------------------+ + |***Spark| Spark| + | Spark**| Spark**| + | *Spark| Spark| + +--------+--------------------------+ """ - return _invoke_function_over_columns("ltrim", col) + if trim is not None: + return _invoke_function_over_columns("ltrim", col, trim) + else: + return _invoke_function_over_columns("ltrim", col) @_try_remote_functions -def rtrim(col: "ColumnOrName") -> Column: +def rtrim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: """ Trim the spaces from right end for the specified string value. @@ -10961,6 +10984,10 @@ def rtrim(col: "ColumnOrName") -> Column: ---------- col : :class:`~pyspark.sql.Column` or str target column to work on. + trim : :class:`~pyspark.sql.Column` or str, optional + The trim string characters to trim, the default value is a single space + + .. versionadded:: 4.0.0 Returns ------- @@ -10969,21 +10996,40 @@ def rtrim(col: "ColumnOrName") -> Column: Examples -------- + Example 1: Trim the spaces + + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([" Spark", "Spark ", " Spark"], "STRING") - >>> df.select(rtrim("value").alias("r")).withColumn("length", length("r")).show() - +--------+------+ - | r|length| - +--------+------+ - | Spark| 8| - | Spark| 5| - | Spark| 6| - +--------+------+ + >>> df.select("*", sf.rtrim("value")).show() + +--------+------------+ + | value|rtrim(value)| + +--------+------------+ + | Spark| Spark| + | Spark | Spark| + | Spark| Spark| + +--------+------------+ + + Example 2: Trim specified characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame(["***Spark", "Spark**", "*Spark"], "STRING") + >>> df.select("*", sf.rtrim("value", sf.lit("*"))).show() + +--------+---------------------------+ + | value|TRIM(TRAILING * FROM value)| + +--------+---------------------------+ + |***Spark| ***Spark| + | Spark**| Spark| + | *Spark| *Spark| + +--------+---------------------------+ """ - return _invoke_function_over_columns("rtrim", col) + if trim is not None: + return _invoke_function_over_columns("rtrim", col, trim) + else: + return _invoke_function_over_columns("rtrim", col) @_try_remote_functions -def trim(col: "ColumnOrName") -> Column: +def trim(col: "ColumnOrName", trim: Optional["ColumnOrName"] = None) -> Column: """ Trim the spaces from both ends for the specified string column. @@ -10996,6 +11042,10 @@ def trim(col: "ColumnOrName") -> Column: ---------- col : :class:`~pyspark.sql.Column` or str target column to work on. + trim : :class:`~pyspark.sql.Column` or str, optional + The trim string characters to trim, the default value is a single space + + .. versionadded:: 4.0.0 Returns ------- @@ -11004,17 +11054,36 @@ def trim(col: "ColumnOrName") -> Column: Examples -------- + Example 1: Trim the spaces + + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([" Spark", "Spark ", " Spark"], "STRING") - >>> df.select(trim("value").alias("r")).withColumn("length", length("r")).show() - +-----+------+ - | r|length| - +-----+------+ - |Spark| 5| - |Spark| 5| - |Spark| 5| - +-----+------+ - """ - return _invoke_function_over_columns("trim", col) + >>> df.select("*", sf.trim("value")).show() + +--------+-----------+ + | value|trim(value)| + +--------+-----------+ + | Spark| Spark| + | Spark | Spark| + | Spark| Spark| + +--------+-----------+ + + Example 2: Trim specified characters + + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame(["***Spark", "Spark**", "*Spark"], "STRING") + >>> df.select("*", sf.trim("value", sf.lit("*"))).show() + +--------+-----------------------+ + | value|TRIM(BOTH * FROM value)| + +--------+-----------------------+ + |***Spark| Spark| + | Spark**| Spark| + | *Spark| Spark| + +--------+-----------------------+ + """ + if trim is not None: + return _invoke_function_over_columns("trim", col, trim) + else: + return _invoke_function_over_columns("trim", col) @_try_remote_functions diff --git a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala index e6fd06f2ec632..4838bc5298bb3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/functions.scala @@ -4061,7 +4061,14 @@ object functions { * @group string_funcs * @since 2.3.0 */ - def ltrim(e: Column, trimString: String): Column = Column.fn("ltrim", lit(trimString), e) + def ltrim(e: Column, trimString: String): Column = ltrim(e, lit(trimString)) + + /** + * Trim the specified character string from left end for the specified string column. + * @group string_funcs + * @since 4.0.0 + */ + def ltrim(e: Column, trim: Column): Column = Column.fn("ltrim", trim, e) /** * Calculates the byte length for the specified string column. @@ -4258,7 +4265,14 @@ object functions { * @group string_funcs * @since 2.3.0 */ - def rtrim(e: Column, trimString: String): Column = Column.fn("rtrim", lit(trimString), e) + def rtrim(e: Column, trimString: String): Column = rtrim(e, lit(trimString)) + + /** + * Trim the specified character string from right end for the specified string column. + * @group string_funcs + * @since 4.0.0 + */ + def rtrim(e: Column, trim: Column): Column = Column.fn("rtrim", trim, e) /** * Returns the soundex code for the specified expression. @@ -4444,7 +4458,14 @@ object functions { * @group string_funcs * @since 2.3.0 */ - def trim(e: Column, trimString: String): Column = Column.fn("trim", lit(trimString), e) + def trim(e: Column, trimString: String): Column = trim(e, lit(trimString)) + + /** + * Trim the specified character from both ends for the specified string column. + * @group string_funcs + * @since 4.0.0 + */ + def trim(e: Column, trim: Column): Column = Column.fn("trim", trim, e) /** * Converts a string column to upper case.