You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "beliefer (via GitHub)" <gi...@apache.org> on 2023/09/02 12:12:14 UTC

[GitHub] [spark] beliefer commented on a diff in pull request #42775: [SPARK-45052][SQL][PYTHON][CONNECT] Make function aliases output column name consistent with SQL

beliefer commented on code in PR #42775:
URL: https://github.com/apache/spark/pull/42775#discussion_r1313816119


##########
connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -987,7 +987,7 @@ object functions {
    * @group agg_funcs
    * @since 3.5.0
    */
-  def std(e: Column): Column = stddev(e)
+  def std(e: Column): Column = Column.fn("std", e)

Review Comment:
   Looks good.



##########
sql/core/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -575,7 +575,7 @@ object functions {
    * @group agg_funcs
    * @since 3.5.0
    */
-  def first_value(e: Column): Column = first(e)
+  def first_value(e: Column): Column = call_function("first_value", e)

Review Comment:
   Seems good.



##########
sql/core/src/main/scala/org/apache/spark/sql/functions.scala:
##########
@@ -1052,15 +1049,15 @@ object functions {
    * @group agg_funcs
    * @since 3.5.0
    */
-  def std(e: Column): Column = stddev(e)
+  def std(e: Column): Column = call_function("std", e)
 
   /**
    * Aggregate function: alias for `stddev_samp`.
    *
    * @group agg_funcs
    * @since 1.6.0
    */
-  def stddev(e: Column): Column = withAggregateFunction { StddevSamp(e.expr) }
+  def stddev(e: Column): Column = call_function("stddev", e)

Review Comment:
   This one may is a break change. Even if the behavior is more suitable.



##########
python/pyspark/sql/functions.py:
##########
@@ -2385,25 +2416,54 @@ def signum(col: "ColumnOrName") -> Column:
 
     Examples
     --------
-    >>> df = spark.range(1)
-    >>> df.select(signum(lit(-5))).show()
-    +----------+
-    |SIGNUM(-5)|
-    +----------+
-    |      -1.0|
-    +----------+
-
-    >>> df.select(signum(lit(6))).show()
-    +---------+
-    |SIGNUM(6)|
-    +---------+
-    |      1.0|
-    +---------+
+    >>> import pyspark.sql.functions as sf
+    >>> spark.range(1).select(
+    ...     sf.signum(sf.lit(-5)),
+    ...     sf.signum(sf.lit(6))
+    ... ).show()
+    +----------+---------+
+    |SIGNUM(-5)|SIGNUM(6)|
+    +----------+---------+
+    |      -1.0|      1.0|
+    +----------+---------+
     """
     return _invoke_function_over_columns("signum", col)
 
 
-sign = signum
+@try_remote_functions
+def sign(col: "ColumnOrName") -> Column:
+    """
+    Computes the signum of the given value.
+
+    .. versionadded:: 1.4.0
+
+    .. versionchanged:: 3.4.0

Review Comment:
   3.5.0 ?



##########
python/pyspark/sql/functions.py:
##########
@@ -2228,14 +2258,15 @@ def negative(col: "ColumnOrName") -> Column:
 
     Examples
     --------
-    >>> spark.range(3).select(negative("id").alias("n")).show()
-    +---+
-    |  n|
-    +---+
-    |  0|
-    | -1|
-    | -2|
-    +---+
+    >>> import pyspark.sql.functions as sf
+    >>> spark.range(3).select(sf.negative("id")).show()

Review Comment:
   Good change.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org