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 2021/09/01 09:26:04 UTC

[GitHub] [spark] xinrong-databricks commented on a change in pull request #33858: [SPARK-36402][PYTHON] Implement Series.combine

xinrong-databricks commented on a change in pull request #33858:
URL: https://github.com/apache/spark/pull/33858#discussion_r699660235



##########
File path: python/pyspark/pandas/series.py
##########
@@ -4425,6 +4427,127 @@ def replace(
 
         return self._with_new_scol(current)  # TODO: dtype?
 
+    def combine(
+        self,
+        other: "Series",
+        func: Callable,
+        fill_value: Optional[Any] = None,
+        return_type: Optional[Union[str, DataType]] = "string",
+    ) -> "Series":
+        """
+        Combine the Series with a Series or scalar according to `func`.
+        Combine the Series and `other` using `func` to perform elementwise
+        selection for combined Series.
+        `fill_value` is assumed when value is missing at some index
+        from one of the two objects being combined.
+
+        Parameters
+        ----------
+        other : Series or scalar
+            The value(s) to be combined with the `Series`.
+        func : function
+            Function that takes two scalars as inputs and returns an element.
+        fill_value : scalar, optional
+            The value to assume when an index is missing from
+            one Series or the other. The default specifies to use the
+            appropriate NaN value for the underlying dtype of the Series.
+        return_type : :class:`pyspark.sql.types.DataType` or str
+            the return type of the output Series. The value can be either a
+            :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
+
+        Returns
+        -------
+        Series
+            The result of combining the Series with the other object.
+
+        See Also
+        --------
+        Series.combine_first : Combine Series values, choosing the calling
+            Series' values first.
+
+        Examples
+        --------
+        Consider 2 Datasets ``s1`` and ``s2`` containing
+        highest clocked speeds of different birds.
+
+        >>> from pyspark.pandas.config import set_option, reset_option
+        >>> set_option("compute.ops_on_diff_frames", True)
+        >>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+        >>> s1
+        falcon    330.0
+        eagle     160.0
+        dtype: float64
+        >>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+        >>> s2
+        falcon    345.0
+        eagle     200.0
+        duck       30.0
+        dtype: float64
+
+        Now, to combine the two datasets and view the highest speeds
+        of the birds across the two datasets
+
+        >>> s1.combine(s2, max, return_type="double")
+        duck        NaN
+        eagle     200.0
+        falcon    345.0
+        dtype: float64
+
+        In the previous example, the resulting value for duck is missing,
+        because the maximum of a NaN and a float is a NaN.
+        So, in the example, we set ``fill_value=0``,
+        so the maximum value returned will be the value from some dataset.
+
+        >>> s1.combine(s2, max, fill_value=0, return_type="double")
+        duck       30.0
+        eagle     200.0
+        falcon    345.0
+        dtype: float64
+        >>> reset_option("compute.ops_on_diff_frames")
+        """
+        if not isinstance(other, Series) and not np.isscalar(other):
+            raise TypeError("unsupported type: %s" % type(other))
+
+        if np.isscalar(other):
+            combined = DataFrame(
+                InternalFrame(
+                    spark_frame=self._internal.spark_frame.withColumn("__other", SF.lit(other)),

Review comment:
       Shall we verify `__other` by https://github.com/apache/spark/blob/master/python/pyspark/pandas/utils.py#L795?

##########
File path: python/pyspark/pandas/series.py
##########
@@ -4425,6 +4427,127 @@ def replace(
 
         return self._with_new_scol(current)  # TODO: dtype?
 
+    def combine(
+        self,
+        other: "Series",
+        func: Callable,
+        fill_value: Optional[Any] = None,
+        return_type: Optional[Union[str, DataType]] = "string",
+    ) -> "Series":
+        """
+        Combine the Series with a Series or scalar according to `func`.
+        Combine the Series and `other` using `func` to perform elementwise
+        selection for combined Series.
+        `fill_value` is assumed when value is missing at some index
+        from one of the two objects being combined.
+
+        Parameters
+        ----------
+        other : Series or scalar
+            The value(s) to be combined with the `Series`.
+        func : function
+            Function that takes two scalars as inputs and returns an element.
+        fill_value : scalar, optional
+            The value to assume when an index is missing from
+            one Series or the other. The default specifies to use the
+            appropriate NaN value for the underlying dtype of the Series.
+        return_type : :class:`pyspark.sql.types.DataType` or str
+            the return type of the output Series. The value can be either a
+            :class:`pyspark.sql.types.DataType` object or a DDL-formatted type string.
+
+        Returns
+        -------
+        Series
+            The result of combining the Series with the other object.
+
+        See Also
+        --------
+        Series.combine_first : Combine Series values, choosing the calling
+            Series' values first.
+
+        Examples
+        --------
+        Consider 2 Datasets ``s1`` and ``s2`` containing
+        highest clocked speeds of different birds.
+
+        >>> from pyspark.pandas.config import set_option, reset_option
+        >>> set_option("compute.ops_on_diff_frames", True)
+        >>> s1 = ps.Series({'falcon': 330.0, 'eagle': 160.0})
+        >>> s1
+        falcon    330.0
+        eagle     160.0
+        dtype: float64
+        >>> s2 = ps.Series({'falcon': 345.0, 'eagle': 200.0, 'duck': 30.0})
+        >>> s2
+        falcon    345.0
+        eagle     200.0
+        duck       30.0
+        dtype: float64
+
+        Now, to combine the two datasets and view the highest speeds
+        of the birds across the two datasets
+
+        >>> s1.combine(s2, max, return_type="double")
+        duck        NaN
+        eagle     200.0
+        falcon    345.0
+        dtype: float64
+
+        In the previous example, the resulting value for duck is missing,
+        because the maximum of a NaN and a float is a NaN.
+        So, in the example, we set ``fill_value=0``,
+        so the maximum value returned will be the value from some dataset.
+
+        >>> s1.combine(s2, max, fill_value=0, return_type="double")
+        duck       30.0
+        eagle     200.0
+        falcon    345.0
+        dtype: float64
+        >>> reset_option("compute.ops_on_diff_frames")
+        """
+        if not isinstance(other, Series) and not np.isscalar(other):
+            raise TypeError("unsupported type: %s" % type(other))
+
+        if np.isscalar(other):
+            combined = DataFrame(
+                InternalFrame(
+                    spark_frame=self._internal.spark_frame.withColumn("__other", SF.lit(other)),
+                    index_spark_columns=self._internal.index_spark_columns,
+                )
+            )
+        else:
+            combined = (
+                self._psdf[self._column_label, other._column_label]
+                if same_anchor(self, other)
+                else combine_frames(self.to_frame(), other.to_frame())
+            )
+
+        sdf = combined._internal.spark_frame
+        if fill_value is not None:
+            sdf = sdf.fillna(fill_value)
+
+        @udf

Review comment:
       What's the concern about not using pandas_udf here?




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