Return type for _ExpandIntoRanges DoFn should be Iterable. (#22548)
The following fails otherwise:
```
import apache_beam as beam
from apache_beam.io import textio
with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
runtime_type_check=True, type_check_additional='all')) as p:
p | beam.Create(['/bin/sh']) | textio.ReadAllFromText()
```
```
apache_beam.typehints.typehints.CompositeTypeHintError: Union[FileMetadata, OffsetRange] type-constraint violated. Expected an instance of one of: ('FileMetadata', 'OffsetRange'), received tuple instead.
```
diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py
index 6592a0c..e2cd268 100644
--- a/sdks/python/apache_beam/io/filebasedsource.py
+++ b/sdks/python/apache_beam/io/filebasedsource.py
@@ -29,6 +29,7 @@
# pytype: skip-file
from typing import Callable
+from typing import Iterable
from typing import Tuple
from typing import Union
@@ -347,7 +348,7 @@
self._compression_type = compression_type
def process(self, element: Union[str, FileMetadata], *args,
- **kwargs) -> Tuple[FileMetadata, OffsetRange]:
+ **kwargs) -> Iterable[Tuple[FileMetadata, OffsetRange]]:
if isinstance(element, FileMetadata):
metadata_list = [element]
else: