From 1ab280e5cfc01c5db45f8484b0640035ec5b1843 Mon Sep 17 00:00:00 2001 From: Martin Bomio Date: Wed, 19 Feb 2025 04:12:40 +0000 Subject: [PATCH] feat: allow normalizer to be used in append mode Signed-off-by: Martin Bomio --- python/ray/data/preprocessors/normalizer.py | 33 ++++++++++++++++--- .../tests/preprocessors/test_normalizer.py | 29 ++++++++++++++-- 2 files changed, 55 insertions(+), 7 deletions(-) diff --git a/python/ray/data/preprocessors/normalizer.py b/python/ray/data/preprocessors/normalizer.py index 430bf6ec6c09a..f94f22b8fcc16 100644 --- a/python/ray/data/preprocessors/normalizer.py +++ b/python/ray/data/preprocessors/normalizer.py @@ -1,4 +1,4 @@ -from typing import List +from typing import List, Optional import numpy as np import pandas as pd @@ -65,11 +65,24 @@ class Normalizer(Preprocessor): 0 1.0 1.0 0 1 1.0 0.0 1 + :class:`Normalizer` can also be used in append mode by providing the + name of the output_columns that should hold the normalized values. + + >>> preprocessor = Normalizer(columns=["X1", "X2"], output_columns=["X1_normalized", "X2_normalized"]) + >>> preprocessor.fit_transform(ds).to_pandas() # doctest: +SKIP + X1 X2 X3 X1_normalized X2_normalized + 0 1 1 0 0.707107 0.707107 + 1 1 0 1 1.000000 0.000000 + Args: columns: The columns to scale. For each row, these colmumns are scaled to unit-norm. norm: The norm to use. The supported values are ``"l1"``, ``"l2"``, or ``"max"``. Defaults to ``"l2"``. + output_columns: The names of the transformed columns. If None, the transformed + columns will be the same as the input columns. If not None, the length of + ``output_columns`` must match the length of ``columns``, othwerwise an error + will be raised. Raises: ValueError: if ``norm`` is not ``"l1"``, ``"l2"``, or ``"max"``. @@ -83,7 +96,13 @@ class Normalizer(Preprocessor): _is_fittable = False - def __init__(self, columns: List[str], norm="l2"): + def __init__( + self, + columns: List[str], + norm="l2", + *, + output_columns: Optional[List[str]] = None, + ): self.columns = columns self.norm = norm @@ -93,14 +112,20 @@ def __init__(self, columns: List[str], norm="l2"): f"Supported values are: {self._norm_fns.keys()}" ) + self.output_columns = Preprocessor._derive_and_validate_output_columns( + columns, output_columns + ) + def _transform_pandas(self, df: pd.DataFrame): columns = df.loc[:, self.columns] column_norms = self._norm_fns[self.norm](columns) - df.loc[:, self.columns] = columns.div(column_norms, axis=0) + df[self.output_columns] = columns.div(column_norms, axis=0) return df def __repr__(self): return ( - f"{self.__class__.__name__}(columns={self.columns!r}, norm={self.norm!r})" + f"{self.__class__.__name__}(columns={self.columns!r}, " + f"norm={self.norm!r}, " + f"output_columns={self.output_columns!r})" ) diff --git a/python/ray/data/tests/preprocessors/test_normalizer.py b/python/ray/data/tests/preprocessors/test_normalizer.py index 5e2b09d801452..c92cf1662d50c 100644 --- a/python/ray/data/tests/preprocessors/test_normalizer.py +++ b/python/ray/data/tests/preprocessors/test_normalizer.py @@ -27,7 +27,7 @@ def test_normalizer(): {"A": processed_col_a, "B": processed_col_b, "C": processed_col_c} ) - assert out_df.equals(expected_df) + pd.testing.assert_frame_equal(out_df, expected_df, check_like=True) # l1 norm normalizer = Normalizer(["B", "C"], norm="l1") @@ -42,7 +42,7 @@ def test_normalizer(): {"A": processed_col_a, "B": processed_col_b, "C": processed_col_c} ) - assert out_df.equals(expected_df) + pd.testing.assert_frame_equal(out_df, expected_df, check_like=True) # max norm normalizer = Normalizer(["B", "C"], norm="max") @@ -57,7 +57,30 @@ def test_normalizer(): {"A": processed_col_a, "B": processed_col_b, "C": processed_col_c} ) - assert out_df.equals(expected_df) + pd.testing.assert_frame_equal(out_df, expected_df, check_like=True) + + # append mode + with pytest.raises(ValueError): + Normalizer(columns=["B", "C"], output_columns=["B_encoded"]) + + normalizer = Normalizer(["B", "C"], output_columns=["B_normalized", "C_normalized"]) + transformed = normalizer.transform(ds) + out_df = transformed.to_pandas() + + processed_col_a = col_a + processed_col_b = [1 / np.sqrt(5), 0.6, 0.6] + processed_col_c = [2 / np.sqrt(5), 0.8, -0.8] + expected_df = pd.DataFrame.from_dict( + { + "A": col_a, + "B": col_b, + "C": col_c, + "B_normalized": processed_col_b, + "C_normalized": processed_col_c, + } + ) + + pd.testing.assert_frame_equal(out_df, expected_df, check_like=True) if __name__ == "__main__":