tokenizer.py 3.5 KB

1234567891011121314151617181920212223242526272829303132333435363738394041424344454647484950515253545556575859606162636465666768697071727374757677787980818283848586878889909192
  1. from typing import Callable, List, Optional
  2. import pandas as pd
  3. from ray.data.preprocessor import Preprocessor
  4. from ray.data.preprocessors.utils import simple_split_tokenizer
  5. from ray.util.annotations import PublicAPI
  6. @PublicAPI(stability="alpha")
  7. class Tokenizer(Preprocessor):
  8. """Replace each string with a list of tokens.
  9. Examples:
  10. >>> import pandas as pd
  11. >>> import ray
  12. >>> df = pd.DataFrame({"text": ["Hello, world!", "foo bar\\nbaz"]})
  13. >>> ds = ray.data.from_pandas(df) # doctest: +SKIP
  14. The default ``tokenization_fn`` delimits strings using the space character.
  15. >>> from ray.data.preprocessors import Tokenizer
  16. >>> tokenizer = Tokenizer(columns=["text"])
  17. >>> tokenizer.transform(ds).to_pandas() # doctest: +SKIP
  18. text
  19. 0 [Hello,, world!]
  20. 1 [foo, bar\\nbaz]
  21. If the default logic isn't adequate for your use case, you can specify a
  22. custom ``tokenization_fn``.
  23. >>> import string
  24. >>> def tokenization_fn(s):
  25. ... for character in string.punctuation:
  26. ... s = s.replace(character, "")
  27. ... return s.split()
  28. >>> tokenizer = Tokenizer(columns=["text"], tokenization_fn=tokenization_fn)
  29. >>> tokenizer.transform(ds).to_pandas() # doctest: +SKIP
  30. text
  31. 0 [Hello, world]
  32. 1 [foo, bar, baz]
  33. :class:`Tokenizer` can also be used in append mode by providing the
  34. name of the output_columns that should hold the tokenized values.
  35. >>> tokenizer = Tokenizer(columns=["text"], output_columns=["text_tokenized"])
  36. >>> tokenizer.transform(ds).to_pandas() # doctest: +SKIP
  37. text text_tokenized
  38. 0 Hello, world! [Hello,, world!]
  39. 1 foo bar\\nbaz [foo, bar\\nbaz]
  40. Args:
  41. columns: The columns to tokenize.
  42. tokenization_fn: The function used to generate tokens. This function
  43. should accept a string as input and return a list of tokens as
  44. output. If unspecified, the tokenizer uses a function equivalent to
  45. ``lambda s: s.split(" ")``.
  46. output_columns: The names of the transformed columns. If None, the transformed
  47. columns will be the same as the input columns. If not None, the length of
  48. ``output_columns`` must match the length of ``columns``, othwerwise an error
  49. will be raised.
  50. """
  51. _is_fittable = False
  52. def __init__(
  53. self,
  54. columns: List[str],
  55. tokenization_fn: Optional[Callable[[str], List[str]]] = None,
  56. output_columns: Optional[List[str]] = None,
  57. ):
  58. super().__init__()
  59. self.columns = columns
  60. # TODO(matt): Add a more robust default tokenizer.
  61. self.tokenization_fn = tokenization_fn or simple_split_tokenizer
  62. self.output_columns = Preprocessor._derive_and_validate_output_columns(
  63. columns, output_columns
  64. )
  65. def _transform_pandas(self, df: pd.DataFrame):
  66. def column_tokenizer(s: pd.Series):
  67. return s.map(self.tokenization_fn)
  68. df[self.output_columns] = df.loc[:, self.columns].transform(column_tokenizer)
  69. return df
  70. def __repr__(self):
  71. name = getattr(self.tokenization_fn, "__name__", self.tokenization_fn)
  72. return (
  73. f"{self.__class__.__name__}(columns={self.columns!r}, "
  74. f"tokenization_fn={name}, output_columns={self.output_columns!r})"
  75. )