Skip to content

Commit

Permalink
[data/preprocessors] feat: allow tokenizer to execute in append mode (r…
Browse files Browse the repository at this point in the history
…ay-project#50848)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

This is part of ray-project#48133.
Continuing the approach taken in
ray-project#49426, make tokenizer work in
append mode

## Related issue number

ray-project#49426

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run `scripts/format.sh` to lint the changes in this PR.
- [x] I've included any doc changes needed for
https://docs.ray.io/en/master/.
- [x] I've added any new APIs to the API Reference. For example, if I
added a
method in Tune, I've added it in `doc/source/tune/api/` under the
           corresponding `.rst` file.
- [x] I've made sure the tests are passing. Note that there might be a
few flaky tests, see the recent failures at https://flakey-tests.ray.io/
- Testing Strategy
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

Signed-off-by: Martin Bomio <martinbomio@spotify.com>
  • Loading branch information
martinbomio authored and Michaelhess17 committed Mar 3, 2025
1 parent 4affb74 commit b7d11d2
Show file tree
Hide file tree
Showing 2 changed files with 71 additions and 3 deletions.
21 changes: 19 additions & 2 deletions python/ray/data/preprocessors/tokenizer.py
Original file line number Diff line number Diff line change
Expand Up @@ -40,12 +40,25 @@ class Tokenizer(Preprocessor):
0 [Hello, world]
1 [foo, bar, baz]
:class:`Tokenizer` can also be used in append mode by providing the
name of the output_columns that should hold the tokenized values.
>>> tokenizer = Tokenizer(columns=["text"], output_columns=["text_tokenized"])
>>> tokenizer.transform(ds).to_pandas() # doctest: +SKIP
text text_tokenized
0 Hello, world! [Hello,, world!]
1 foo bar\\nbaz [foo, bar\\nbaz]
Args:
columns: The columns to tokenize.
tokenization_fn: The function used to generate tokens. This function
should accept a string as input and return a list of tokens as
output. If unspecified, the tokenizer uses a function equivalent to
``lambda s: s.split(" ")``.
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.
"""

_is_fittable = False
Expand All @@ -54,21 +67,25 @@ def __init__(
self,
columns: List[str],
tokenization_fn: Optional[Callable[[str], List[str]]] = None,
output_columns: Optional[List[str]] = None,
):
self.columns = columns
# TODO(matt): Add a more robust default tokenizer.
self.tokenization_fn = tokenization_fn or simple_split_tokenizer
self.output_columns = Preprocessor._derive_and_validate_output_columns(
columns, output_columns
)

def _transform_pandas(self, df: pd.DataFrame):
def column_tokenizer(s: pd.Series):
return s.map(self.tokenization_fn)

df.loc[:, self.columns] = df.loc[:, self.columns].transform(column_tokenizer)
df[self.output_columns] = df.loc[:, self.columns].transform(column_tokenizer)
return df

def __repr__(self):
name = getattr(self.tokenization_fn, "__name__", self.tokenization_fn)
return (
f"{self.__class__.__name__}(columns={self.columns!r}, "
f"tokenization_fn={name})"
f"tokenization_fn={name}, output_columns={self.output_columns!r})"
)
53 changes: 52 additions & 1 deletion python/ray/data/tests/preprocessors/test_tokenizer.py
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,58 @@ def test_tokenizer():
]
expected_df = pd.DataFrame.from_dict({"A": processed_col_a, "B": processed_col_b})

assert out_df.equals(expected_df)
pd.testing.assert_frame_equal(out_df, expected_df, check_like=True)

# Test append mode
with pytest.raises(
ValueError, match="The length of columns and output_columns must match."
):
Tokenizer(columns=["A", "B"], output_columns=["A_tokenized"])

tokenizer = Tokenizer(
columns=["A", "B"], output_columns=["A_tokenized", "B_tokenized"]
)
transformed = tokenizer.transform(ds)
out_df = transformed.to_pandas()
print(out_df)
expected_df = pd.DataFrame.from_dict(
{
"A": col_a,
"B": col_b,
"A_tokenized": processed_col_a,
"B_tokenized": processed_col_b,
}
)

pd.testing.assert_frame_equal(out_df, expected_df, check_like=True)

# Test custom tokenization function
def custom_tokenizer(s: str) -> list:
return s.replace("banana", "fruit").split()

tokenizer = Tokenizer(
columns=["A", "B"],
tokenization_fn=custom_tokenizer,
output_columns=["A_custom", "B_custom"],
)
transformed = tokenizer.transform(ds)
out_df = transformed.to_pandas()

custom_processed_col_a = [["this", "is", "a", "test"], ["apple"]]
custom_processed_col_b = [
["the", "quick", "brown", "fox", "jumps", "over", "the", "lazy", "dog"],
["fruit", "fruit"],
]
expected_df = pd.DataFrame.from_dict(
{
"A": col_a,
"B": col_b,
"A_custom": custom_processed_col_a,
"B_custom": custom_processed_col_b,
}
)

pd.testing.assert_frame_equal(out_df, expected_df, check_like=True)


if __name__ == "__main__":
Expand Down

0 comments on commit b7d11d2

Please sign in to comment.