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/02/04 10:38:41 UTC

[GitHub] [spark] zero323 commented on pull request #32431: [SPARK-35173][SQL][PYTHON] Add multiple columns adding support

zero323 commented on pull request #32431:
URL: https://github.com/apache/spark/pull/32431#issuecomment-1029850928


   Since we are revisiting it have a counter-proposal ‒ instead exposing new API, let's just improve UX of what we already have. For example, if we tweak `select` to support keyword arguments like this:
   
   ```patch
   diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
   index ee68865c98..00a7a4543e 100644
   --- a/python/pyspark/sql/dataframe.py
   +++ b/python/pyspark/sql/dataframe.py
   @@ -1941,14 +1941,18 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
            return Column(jc)
    
        @overload
   -    def select(self, *cols: "ColumnOrName") -> "DataFrame":
   +    def select(self, *cols: "ColumnOrName", **acols: Column) -> "DataFrame":
            ...
    
        @overload
   -    def select(self, __cols: Union[List[Column], List[str]]) -> "DataFrame":
   +    def select(self, __cols: Union[List[Column], List[str]], **acols: Column) -> "DataFrame":
            ...
    
   -    def select(self, *cols: "ColumnOrName") -> "DataFrame":  # type: ignore[misc]
   +    def select(  # type: ignore[misc]
   +        self,
   +        *cols: "ColumnOrName",
   +        **namedCols: Column,
   +    ) -> "DataFrame":
            """Projects a set of expressions and returns a new :class:`DataFrame`.
    
            .. versionadded:: 1.3.0
   @@ -1959,6 +1963,8 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
                column names (string) or expressions (:class:`Column`).
                If one of the column names is '*', that column is expanded to include all columns
                in the current :class:`DataFrame`.
   +        namedCols : :class:`Column`
   +            exprasssions select under given alaias.
    
            Examples
            --------
   @@ -1968,8 +1974,17 @@ class DataFrame(PandasMapOpsMixin, PandasConversionMixin):
            [Row(name='Alice', age=2), Row(name='Bob', age=5)]
            >>> df.select(df.name, (df.age + 10).alias('age')).collect()
            [Row(name='Alice', age=12), Row(name='Bob', age=15)]
   -        """
   -        jdf = self._jdf.select(self._jcols(*cols))
   +        >>> df.select(
   +        ...     "age",
   +        ...     name_lower=lower("name"),
   +        ...     name_upper=upper("name"),
   +        ...     age_plus_one=col("age") + 1,
   +        ... ).limit(1).collect()
   +        [Row(age=2, name_lower='alice', name_upper='ALICE', age_plus_one=3)]
   +        """
   +        if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
   +            cols = cols[0]  # type: ignore[assignment]
   +        jdf = self._jdf.select(self._jcols(*cols, *[c.alias(a) for a, c in namedCols.items()]))
            return DataFrame(jdf, self.sql_ctx)
    
        @overload
   
   ```
   
   will be able to use it like this:
   
   ```
   df = spark.range(10).select(rand(42).alias("id"))
   
   df.select(
       "id", plus_one=col("id") + 1, times_two=col("id") * 2, log_id = log("id")
   ).show()
   ```


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