You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/11/29 05:42:18 UTC

[GitHub] [spark] HyukjinKwon commented on a diff in pull request #38806: [SPARK-41268][CONNECT][PYTHON] Refactor "Column" for API Compatibility

HyukjinKwon commented on code in PR #38806:
URL: https://github.com/apache/spark/pull/38806#discussion_r1034320960


##########
python/pyspark/sql/connect/column.py:
##########
@@ -314,3 +323,70 @@ def to_plan(self, session: "SparkConnectClient") -> proto.Expression:
 
     def __str__(self) -> str:
         return f"({self._op} ({', '.join([str(x) for x in self._args])}))"
+
+
+class Column(object):
+    """
+    A column in a DataFrame. Column can refer to different things based on the
+    wrapped expression. Some common examples include attribute references, functions,
+    literals, etc.
+
+    .. versionadded:: 3.4.0
+    """
+
+    def __init__(self, expr: Expression) -> None:
+        self._expr = expr
+
+    __gt__ = _bin_op(">")
+    __lt__ = _bin_op("<")
+    __add__ = _bin_op("+")
+    __sub__ = _bin_op("-")
+    __mul__ = _bin_op("*")
+    __div__ = _bin_op("/")
+    __truediv__ = _bin_op("/")
+    __mod__ = _bin_op("%")
+    __radd__ = _bin_op("+", reverse=True)
+    __rsub__ = _bin_op("-", reverse=True)
+    __rmul__ = _bin_op("*", reverse=True)
+    __rdiv__ = _bin_op("/", reverse=True)
+    __rtruediv__ = _bin_op("/", reverse=True)
+    __pow__ = _bin_op("pow")
+    __rpow__ = _bin_op("pow", reverse=True)
+    __ge__ = _bin_op(">=")
+    __le__ = _bin_op("<=")
+    # __eq__ = _bin_op("==")  # ignore [assignment]
+
+    def __eq__(self, other: Any) -> "Column":  # type: ignore[override]
+        """Returns a binary expression with the current column as the left
+        side and the other expression as the right side.
+        """
+        from pyspark.sql.connect._typing import PrimitiveType
+        from pyspark.sql.connect.functions import lit
+
+        if isinstance(other, get_args(PrimitiveType)):
+            other = lit(other)
+        return scalar_function("==", self, other)
+
+    def to_plan(self, session: "SparkConnectClient") -> proto.Expression:
+        return self._expr.to_plan(session)
+
+    def alias(self, *alias: str, **kwargs: Any) -> "Column":
+        return Column(self._expr.alias(*alias, **kwargs))
+
+    def desc(self) -> "Column":
+        return Column(self._expr.desc())
+
+    def asc(self) -> "Column":
+        return Column(self._expr.asc())
+
+    def ascending(self) -> bool:
+        return self._expr.ascending()
+
+    def nullsLast(self) -> bool:

Review Comment:
   Hm, PySpark's `Column` doesn't have this. In fact, `SortOrder` doesn't exist. Should probably match it with that.



-- 
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