diff --git a/CHANGES.md b/CHANGES.md index 0932d8ed8c34..e04d6c5852b6 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -68,7 +68,7 @@ ## Breaking Changes -* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). +* Python SDK CoGroupByKey outputs an iterable allowing for arbitrarily large results. [#21556](https://github.com/apache/beam/issues/21556) Beam users may see an error on transforms downstream from CoGroupByKey. Users must change methods expecting a List to expect an Iterable going forward. See [document](https://docs.google.com/document/d/1RIzm8-g-0CyVsPb6yasjwokJQFoKHG4NjRUcKHKINu0) for information and fixes. ## Deprecations diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py index 13d68df3715f..cb4b86245e00 100644 --- a/sdks/python/apache_beam/transforms/util.py +++ b/sdks/python/apache_beam/transforms/util.py @@ -198,8 +198,7 @@ def expand(self, pcolls): input_value_types.append(value_type) output_key_type = typehints.Union[tuple(input_key_types)] iterable_input_value_types = tuple( - # TODO: Change List[t] to Iterable[t] - typehints.List[t] for t in input_value_types) + typehints.Iterable[t] for t in input_value_types) output_value_type = typehints.Dict[ str, typehints.Union[iterable_input_value_types or [typehints.Any]]]