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/05/25 05:59:08 UTC

[GitHub] [spark] zhengruifeng commented on a diff in pull request #36660: [SPARK-39284][PS] Implement Groupby.mad

zhengruifeng commented on code in PR #36660:
URL: https://github.com/apache/spark/pull/36660#discussion_r881246146


##########
python/pyspark/pandas/groupby.py:
##########
@@ -759,6 +759,99 @@ def skew(scol: Column) -> Column:
             bool_to_numeric=True,
         )
 
+    # TODO: 'axis', 'skipna', 'level' parameter should be implemented.
+    def mad(self) -> FrameLike:
+        """
+        Compute mean absolute deviation of groups, excluding missing values.
+
+        .. versionadded:: 3.4.0
+
+        Examples
+        --------
+        >>> df = ps.DataFrame({"A": [1, 2, 1, 1], "B": [True, False, False, True],
+        ...                    "C": [3, 4, 3, 4], "D": ["a", "b", "b", "a"]})
+
+        >>> df.groupby("A").mad()
+                  B         C
+        A
+        1  0.444444  0.444444
+        2  0.000000  0.000000
+
+        >>> df.B.groupby(df.A).mad()
+        A
+        1    0.444444
+        2    0.000000
+        Name: B, dtype: float64
+
+        See Also
+        --------
+        pyspark.pandas.Series.groupby
+        pyspark.pandas.DataFrame.groupby
+        """
+
+        groupkey_names = [SPARK_INDEX_NAME_FORMAT(i) for i in range(len(self._groupkeys))]
+        groupkey_scols = [s.alias(name) for s, name in zip(self._groupkeys_scols, groupkey_names)]
+
+        agg_columns = []
+        for psser in self._agg_columns:
+            if isinstance(psser.spark.data_type, BooleanType):
+                agg_columns.append(psser.astype(int))
+            elif isinstance(psser.spark.data_type, NumericType):
+                agg_columns.append(psser)
+
+        sdf = self._psdf._internal.spark_frame.select(
+            *groupkey_scols, *[psser.spark.column for psser in agg_columns]
+        )
+
+        internal = InternalFrame(
+            spark_frame=sdf,
+            index_spark_columns=[scol_for(sdf, col) for col in groupkey_names],
+            index_names=[psser._column_label for psser in self._groupkeys],
+            index_fields=[
+                psser._internal.data_fields[0].copy(name=name)
+                for psser, name in zip(self._groupkeys, groupkey_names)
+            ],
+            data_spark_columns=[
+                scol_for(sdf, psser._internal.data_spark_column_names[0]) for psser in agg_columns
+            ],
+            column_labels=[psser._column_label for psser in agg_columns],
+            data_fields=[psser._internal.data_fields[0] for psser in agg_columns],
+            column_label_names=self._psdf._internal.column_label_names,
+        )
+        psdf: DataFrame = DataFrame(internal)
+
+        if len(psdf._internal.column_labels) > 0:

Review Comment:
   it is not able to directly use `self._reduce_for_stat_function` for `mad` due to `It is not allowed to use a window function inside an aggregate function`.
   
   The implement here refer to `_reduce_for_stat_function`, but has some changes here (apply window function to compute the `mean` for each group)



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