From 0e0cb0b5720e588b45fc5dfa07a6d2640b69bfd7 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Tue, 29 Apr 2025 12:15:21 +0000 Subject: [PATCH 01/19] iobase patch for unbounded PColl and adding streaming writes for AvroIO, ParquetIO and TFRecordIO --- .../examples/unbounded_sinks/__init__.py | 16 + .../unbounded_sinks/generate_event.py | 65 ++++ .../examples/unbounded_sinks/test_write.py | 170 +++++++++++ .../unbounded_sinks/test_write_bounded.py | 100 +++++++ sdks/python/apache_beam/io/avroio.py | 34 ++- sdks/python/apache_beam/io/avroio_test.py | 165 ++++++++++ sdks/python/apache_beam/io/filebasedsink.py | 128 +++++--- sdks/python/apache_beam/io/iobase.py | 281 +++++++++++++++--- sdks/python/apache_beam/io/parquetio.py | 73 ++++- sdks/python/apache_beam/io/parquetio_test.py | 150 ++++++++++ sdks/python/apache_beam/io/textio.py | 32 +- sdks/python/apache_beam/io/textio_test.py | 136 +++++++++ sdks/python/apache_beam/io/tfrecordio.py | 20 +- sdks/python/apache_beam/io/tfrecordio_test.py | 144 +++++++++ 14 files changed, 1411 insertions(+), 103 deletions(-) create mode 100644 sdks/python/apache_beam/examples/unbounded_sinks/__init__.py create mode 100644 sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py create mode 100644 sdks/python/apache_beam/examples/unbounded_sinks/test_write.py create mode 100644 sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/__init__.py b/sdks/python/apache_beam/examples/unbounded_sinks/__init__.py new file mode 100644 index 000000000000..cce3acad34a4 --- /dev/null +++ b/sdks/python/apache_beam/examples/unbounded_sinks/__init__.py @@ -0,0 +1,16 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py new file mode 100644 index 000000000000..7a429c1460b8 --- /dev/null +++ b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py @@ -0,0 +1,65 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import apache_beam as beam +from apache_beam.testing.test_stream import TestStream +from datetime import datetime +import pytz +import pyarrow + + +class GenerateEvent(beam.PTransform): + + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + my_schema = pyarrow.schema( + [pyarrow.field('age', pyarrow.int64())], + metadata = {"age":"the age int64"} + ) + #elem = [pyarrow.Table.from_pylist(elemlist, schema=my_schema)] + elem = elemlist + return (input + | TestStream() + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to_infinity()) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py new file mode 100644 index 000000000000..08e110815a20 --- /dev/null +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -0,0 +1,170 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# python -m apache_beam.examples.unbounded_sinks.test_write --runner DirectRunner +# This file contains multiple examples of writing unbounded PCollection to files + +import apache_beam as beam, json, pyarrow +import argparse +import logging +import re +from apache_beam.examples.unbounded_sinks.generate_event import GenerateEvent +from apache_beam.transforms.window import FixedWindows +from apache_beam.transforms.trigger import AccumulationMode +from apache_beam.transforms.trigger import AfterWatermark +from apache_beam.utils.timestamp import Duration +from apache_beam.transforms.util import LogElements +from apache_beam.io.fileio import WriteToFiles +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.runners.runner import PipelineResult + +class CountEvents(beam.PTransform): + def expand(self, events): + return (events + | beam.WindowInto(FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | beam.CombineGlobally(beam.combiners.CountCombineFn()).without_defaults()) + +def run(argv=None, save_main_session=True) -> PipelineResult: + """Main entry point; defines and runs the wordcount pipeline.""" + parser = argparse.ArgumentParser() + known_args, pipeline_args = parser.parse_known_args(argv) + + # We use the save_main_session option because one or more DoFn's in this + # workflow rely on global context (e.g., a module imported at module level). + pipeline_options = PipelineOptions(pipeline_args) + pipeline_options.view_as(SetupOptions).save_main_session = save_main_session + + p = beam.Pipeline(options=pipeline_options) + + output = (p | GenerateEvent.sample_data() + #| 'Count' >> CountEvents() + #| 'Serialize' >> beam.Map(json.dumps) + #| beam.LogElements(prefix='before write ', with_window=False, with_timestamp=True,level=logging.INFO) + ) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix="__output__/ouput_WriteToText", + file_name_suffix=".txt", + #shard_name_template='-V-SSSSS-of-NNNNN', + num_shards=2, + #triggering_frequency=5, + ) + output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + + #FileIO + output5 = ( output + | 'FileIO window' >> beam.WindowInto(FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | 'Serialize' >> beam.Map(json.dumps) + | 'FileIO WriteToFiles' >> WriteToFiles(path="__output__/output_WriteToFiles") + ) + + + #ParquetIO + pyschema = pyarrow.schema( + [('age', pyarrow.int64())] + ) + + output4a = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix="__output__/output_parquet", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".parquet" , + num_shards=2, + triggering_frequency=5, + schema=pyschema + ) + output4a | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet 4a ', with_window=True,level=logging.INFO) + + output4aw = ( output + | 'ParquetIO window' >> beam.WindowInto(FixedWindows(20), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | 'WriteToParquet windowed' >> beam.io.WriteToParquet( + file_path_prefix="__output__/output_parquet", + shard_name_template='-W-SSSSS-of-NNNNN', + file_name_suffix=".parquet" , + num_shards=2, + schema=pyschema + ) + ) + output4aw | 'LogElements after WriteToParquet windowed' >> LogElements(prefix='after WriteToParquet 4aw ', with_window=True,level=logging.INFO) + + + output4b = (output + | 'To PyArrow Table' >> beam.Map(lambda x: pyarrow.Table.from_pylist([x], schema=pyschema)) + | 'WriteToParquetBatched to parquet' >> beam.io.WriteToParquetBatched( + file_path_prefix="__output__/output_parquet_batched", + shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".parquet" , + num_shards=2, + triggering_frequency=5, + schema=pyschema + ) + ) + output4b | 'LogElements after WriteToParquetBatched' >> LogElements(prefix='after WriteToParquetBatched 4b ', with_window=True,level=logging.INFO) + + + #AvroIO + avroschema = { + #'doc': 'A dummy avro file', # a short description + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output5 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix="__output__/output_avro", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".avro" , + num_shards=2, + #triggering_frequency=5, + schema=avroschema + ) + output5 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro 5 ', with_window=True,level=logging.INFO) + + #TFrecordIO + output6 = (output + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) + | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix="__output__/output_tfrecord", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".tfrecord" , + num_shards=2, + triggering_frequency=5 + ) + ) + output6 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord 6 ', with_window=True,level=logging.INFO) + + + # Execute the pipeline and return the result. + result = p.run() + result.wait_until_finish() + return result + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + run() \ No newline at end of file diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py new file mode 100644 index 000000000000..358ebd4dd210 --- /dev/null +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py @@ -0,0 +1,100 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# + +# python -m apache_beam.examples.unbounded_sinks.test_write --runner DirectRunner +# python -m apache_beam.examples.unbounded_sinks.test_write --region us-central1 --project ai-poney --temp_location=gs://poney-us --runner DataflowRunne + +import apache_beam as beam, json, pyarrow +import argparse +import logging +import re +from apache_beam.examples.unbounded_sinks.generate_event import GenerateEvent +from apache_beam.transforms.window import FixedWindows +from apache_beam.transforms.trigger import AccumulationMode +from apache_beam.transforms.trigger import AfterWatermark +from apache_beam.utils.timestamp import Duration +from apache_beam.transforms.util import LogElements +from apache_beam.io.textio import WriteToText +from apache_beam.io.fileio import WriteToFiles +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.runners.runner import PipelineResult + +class CountEvents(beam.PTransform): + def expand(self, events): + return (events + | beam.WindowInto(FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | beam.CombineGlobally(beam.combiners.CountCombineFn()).without_defaults()) + +def run(argv=None, save_main_session=True) -> PipelineResult: + """Main entry point; defines and runs the wordcount pipeline.""" + parser = argparse.ArgumentParser() + known_args, pipeline_args = parser.parse_known_args(argv) + + # We use the save_main_session option because one or more DoFn's in this + # workflow rely on global context (e.g., a module imported at module level). + pipeline_options = PipelineOptions(pipeline_args) + pipeline_options.view_as(SetupOptions).save_main_session = save_main_session + + p = beam.Pipeline(options=pipeline_options) + + output = (p | beam.Create([{'age': 10}, {'age': 20}, {'age': 30}]) + #| beam.CombineGlobally(AverageFn()) + #| 'Serialize' >> beam.Map(json.dumps) + | beam.LogElements(prefix='before write ', with_window=False,level=logging.INFO) ) + #OK in batch + output2 = output | 'Write to text' >> WriteToText(file_path_prefix="__output_batch__/ouput_WriteToText",file_name_suffix=".txt",shard_name_template='-U-SSSSS-of-NNNNN') + output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=False,level=logging.INFO) + + #OK in batch and stream + # output3 = (output | 'Serialize' >> beam.Map(json.dumps) + # | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") + # ) + # output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) + + #KO - ValueError: GroupByKey cannot be applied to an unbounded PCollection with global windowing and a default trigger + # output4 = output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_parquet", + # schema= + # pyarrow.schema( + # [('age', pyarrow.int64())] + # ) + # ) + # output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) + # output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="output", + # schema= + # pyarrow.schema( + # [('cnt', pyarrow.int64()),('json', pyarrow.string())] + # ), + # record_batch_size = 10, + # num_shards=0 + # ) + + + + # Execute the pipeline and return the result. + result = p.run() + result.wait_until_finish() + return result + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + run() \ No newline at end of file diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 8b7958a00b80..e6064a6da44f 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -45,6 +45,8 @@ # pytype: skip-file import ctypes import os +import pytz +import re from functools import partial from typing import Any from typing import Callable @@ -376,7 +378,8 @@ def __init__( num_shards=0, shard_name_template=None, mime_type='application/x-avro', - use_fastavro=True): + use_fastavro=True, + triggering_frequency=None): """Initialize a WriteToAvro transform. Args: @@ -405,6 +408,8 @@ def __init__( supports specifying MIME types. use_fastavro (bool): This flag is left for API backwards compatibility and no longer has an effect. Do not use. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A WriteToAvro transform usable for writing. @@ -417,7 +422,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type) + mime_type, + triggering_frequency) def expand(self, pcoll): if self._schema: @@ -434,6 +440,12 @@ def expand(self, pcoll): records = pcoll | beam.Map( beam_row_to_avro_dict(avro_schema, beam_schema)) self._sink = self._sink_provider(avro_schema) + if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: + # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format + self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + return records | beam.io.iobase.Write(self._sink) def display_data(self): @@ -447,7 +459,8 @@ def _create_avro_sink( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): if "class 'avro.schema" in str(type(schema)): raise ValueError( 'You are using Avro IO with fastavro (default with Beam on ' @@ -460,7 +473,8 @@ def _create_avro_sink( file_name_suffix, num_shards, shard_name_template, - mime_type) + mime_type, + triggering_frequency) class _BaseAvroSink(filebasedsink.FileBasedSink): @@ -473,7 +487,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, file_name_suffix=file_name_suffix, @@ -483,7 +498,8 @@ def __init__( mime_type=mime_type, # Compression happens at the block level using the supplied codec, and # not at the file level. - compression_type=CompressionTypes.UNCOMPRESSED) + compression_type=CompressionTypes.UNCOMPRESSED, + triggering_frequency=triggering_frequency) self._schema = schema self._codec = codec @@ -504,7 +520,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, schema, @@ -512,7 +529,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type) + mime_type, + triggering_frequency) self.file_handle = None def open(self, temp_path): diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 2d25010da486..26b727a160ed 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -16,11 +16,15 @@ # # pytype: skip-file +import glob import json import logging import math import os +import pytz import pytest +import re +import shutil import tempfile import unittest from typing import List, Any @@ -49,14 +53,17 @@ from apache_beam.io.filesystems import FileSystems from apache_beam.options.pipeline_options import StandardOptions from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.transforms.sql import SqlTransform from apache_beam.transforms.userstate import CombiningValueStateSpec +from apache_beam.transforms.util import LogElements from apache_beam.utils.timestamp import Timestamp from apache_beam.typehints import schemas +from datetime import datetime # Import snappy optionally; some tests will be skipped when import fails. try: @@ -646,6 +653,164 @@ def _write_data( self._temp_files.append(f.name) return f.name +class GenerateEvent(beam.PTransform): + + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return (input + | TestStream() + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to_infinity()) + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #AvroIO + avroschema = { + #'doc': 'A dummy avro file', # a short description + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".avro" , + num_shards=num_shards, + schema=avroschema + ) + output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp_xyz/ouput_WriteToAvro-[1614556800.0, 1614556805.0)-00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #AvroIO + avroschema = { + #'doc': 'A dummy avro file', # a short description + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".avro", + shard_name_template=shard_name_template, + num_shards=num_shards, + schema=avroschema + ) + output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, + num_shards = 2, + shard_name_template = '-V-SSSSS-of-NNNNN', + triggering_frequency = 5): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #AvroIO + avroschema = { + #'doc': 'A dummy avro file', # a short description + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'fields': [ # this defines actual keys & their types + {'name': 'age', 'type': 'int'}, + ], + } + output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".txt", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + schema=avroschema + ) + output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + #with 5s window size, the input should be processed by 5 windows with 2 shards per window + self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index eb433bd60583..aa7b6c35b644 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -33,9 +33,12 @@ from apache_beam.options.value_provider import StaticValueProvider from apache_beam.options.value_provider import ValueProvider from apache_beam.options.value_provider import check_accessible +from apache_beam.transforms import window from apache_beam.transforms.display import DisplayDataItem DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' +DEFAULT_WINDOW_SHARD_NAME_TEMPLATE = '-W-SSSSS-of-NNNNN' +DEFAULT_TRIGGERING_FREQUENCY = 60 __all__ = ['FileBasedSink'] @@ -71,7 +74,10 @@ def __init__( *, max_records_per_shard=None, max_bytes_per_shard=None, - skip_if_empty=False): + skip_if_empty=False, + convert_fn=None, + triggering_frequency=None + ): """ Raises: TypeError: if file path parameters are not a :class:`str` or @@ -98,6 +104,8 @@ def __init__( shard_name_template = DEFAULT_SHARD_NAME_TEMPLATE elif shard_name_template == '': num_shards = 1 + if triggering_frequency is None: + triggering_frequency = DEFAULT_TRIGGERING_FREQUENCY if isinstance(file_path_prefix, str): file_path_prefix = StaticValueProvider(str, file_path_prefix) if isinstance(file_name_suffix, str): @@ -106,6 +114,7 @@ def __init__( self.file_name_suffix = file_name_suffix self.num_shards = num_shards self.coder = coder + self.shard_name_template = shard_name_template self.shard_name_format = self._template_to_format(shard_name_template) self.shard_name_glob_format = self._template_to_glob_format( shard_name_template) @@ -114,6 +123,8 @@ def __init__( self.max_records_per_shard = max_records_per_shard self.max_bytes_per_shard = max_bytes_per_shard self.skip_if_empty = skip_if_empty + self.convert_fn = convert_fn + self.triggering_frequency = triggering_frequency def display_data(self): return { @@ -202,25 +213,33 @@ def open_writer(self, init_result, uid): return FileBasedSinkWriter(self, writer_path) @check_accessible(['file_path_prefix', 'file_name_suffix']) - def _get_final_name(self, shard_num, num_shards): + def _get_final_name(self, shard_num, num_shards, w): + if w is None or isinstance(w, window.GlobalWindow): + window_utc = None + else: + window_utc = '['+w.start.to_utc_datetime().isoformat()+', '+w.end.to_utc_datetime().isoformat()+')' return ''.join([ self.file_path_prefix.get(), self.shard_name_format % - dict(shard_num=shard_num, num_shards=num_shards), + dict(shard_num=shard_num, num_shards=num_shards, uuid=(uuid.uuid4()), window=w, window_utc=window_utc), self.file_name_suffix.get() ]) @check_accessible(['file_path_prefix', 'file_name_suffix']) - def _get_final_name_glob(self, num_shards): + def _get_final_name_glob(self, num_shards, w): + if w is None or isinstance(w, window.GlobalWindow): + window_utc = None + else: + window_utc = '['+w.start.to_utc_datetime().isoformat()+', '+w.end.to_utc_datetime().isoformat()+')' return ''.join([ self.file_path_prefix.get(), - self.shard_name_glob_format % dict(num_shards=num_shards), + self.shard_name_glob_format % dict(num_shards=num_shards, uuid=(uuid.uuid4()), window=w, window_utc=window_utc), self.file_name_suffix.get() ]) - def pre_finalize(self, init_result, writer_results): + def pre_finalize(self, init_result, writer_results, window=None): num_shards = len(list(writer_results)) - dst_glob = self._get_final_name_glob(num_shards) + dst_glob = self._get_final_name_glob(num_shards, window) dst_glob_files = [ file_metadata.path for mr in FileSystems.match([dst_glob]) for file_metadata in mr.metadata_list @@ -233,7 +252,7 @@ def pre_finalize(self, init_result, writer_results): self.shard_name_glob_format) FileSystems.delete(dst_glob_files) - def _check_state_for_finalize_write(self, writer_results, num_shards): + def _check_state_for_finalize_write(self, writer_results, num_shards, window=None): """Checks writer output files' states. Returns: @@ -248,7 +267,7 @@ def _check_state_for_finalize_write(self, writer_results, num_shards): return [], [], [], 0 src_glob = FileSystems.join(FileSystems.split(writer_results[0])[0], '*') - dst_glob = self._get_final_name_glob(num_shards) + dst_glob = self._get_final_name_glob(num_shards, window) src_glob_files = set( file_metadata.path for mr in FileSystems.match([src_glob]) for file_metadata in mr.metadata_list) @@ -261,7 +280,7 @@ def _check_state_for_finalize_write(self, writer_results, num_shards): delete_files = [] num_skipped = 0 for shard_num, src in enumerate(writer_results): - final_name = self._get_final_name(shard_num, num_shards) + final_name = self._get_final_name(shard_num, num_shards, window) dst = final_name src_exists = src in src_glob_files dst_exists = dst in dst_glob_files @@ -299,12 +318,12 @@ def _report_sink_lineage(self, dst_glob, dst_files): @check_accessible(['file_path_prefix']) def finalize_write( - self, init_result, writer_results, unused_pre_finalize_results): + self, init_result, writer_results, unused_pre_finalize_results, w=None): writer_results = sorted(writer_results) num_shards = len(writer_results) src_files, dst_files, delete_files, num_skipped = ( - self._check_state_for_finalize_write(writer_results, num_shards)) + self._check_state_for_finalize_write(writer_results, num_shards, w)) num_skipped += len(delete_files) FileSystems.delete(delete_files) num_shards_to_finalize = len(src_files) @@ -322,16 +341,15 @@ def finalize_write( ] if num_shards_to_finalize: - _LOGGER.info( - 'Starting finalize_write threads with num_shards: %d (skipped: %d), ' - 'batches: %d, num_threads: %d', - num_shards_to_finalize, - num_skipped, - len(source_file_batch), - num_threads) + # _LOGGER.info( + # 'Starting finalize_write threads with num_shards: %d (skipped: %d), ' + # 'batches: %d, num_threads: %d', + # num_shards_to_finalize, + # num_skipped, + # len(source_file_batch), + # num_threads) start_time = time.time() - # Use a thread pool for renaming operations. def _rename_batch(batch): """_rename_batch executes batch rename operations.""" source_files, destination_files = batch @@ -355,19 +373,37 @@ def _rename_batch(batch): _LOGGER.debug('Rename successful: %s -> %s', src, dst) return exceptions - exception_batches = util.run_using_threadpool( - _rename_batch, - list(zip(source_file_batch, destination_file_batch)), - num_threads) - - all_exceptions = [ - e for exception_batch in exception_batches for e in exception_batch - ] - if all_exceptions: - raise Exception( - 'Encountered exceptions in finalize_write: %s' % all_exceptions) - - yield from dst_files + if w is None or isinstance(w, window.GlobalWindow): + # bounded input + # Use a thread pool for renaming operations. + + exception_batches = util.run_using_threadpool( + _rename_batch, + list(zip(source_file_batch, destination_file_batch)), + num_threads) + + all_exceptions = [ + e for exception_batch in exception_batches for e in exception_batch + ] + if all_exceptions: + raise Exception( + 'Encountered exceptions in finalize_write: %s' % all_exceptions) + + yield from dst_files + else: + # unbounded input + #batch = list([source_file_batch, destination_file_batch]) + batch = list([src_files, dst_files]) + exception_batches = _rename_batch(batch) + + all_exceptions = [ + e for exception_batch in exception_batches for e in exception_batch + ] + if all_exceptions: + raise Exception( + 'Encountered exceptions in finalize_write: %s' % all_exceptions) + + yield from dst_files _LOGGER.info( 'Renamed %d shards in %.2f seconds.', @@ -384,6 +420,29 @@ def _rename_batch(batch): except IOError: # This error is not serious, we simply log it. _LOGGER.info('Unable to delete file: %s', init_result) + + # return dst_files + + + @staticmethod + def _template_replace_window(shard_name_template): + match = re.search('W+', shard_name_template) + if match: + shard_name_template = shard_name_template.replace( + match.group(0), '%%(window)0%ds' % len(match.group(0))) + match = re.search('V+', shard_name_template) + if match: + shard_name_template = shard_name_template.replace( + match.group(0), '%%(window_utc)0%ds' % len(match.group(0))) + return shard_name_template + + @staticmethod + def _template_replace_uuid(shard_name_template): + match = re.search('U+', shard_name_template) + if match: + shard_name_template = shard_name_template.replace( + match.group(0), '%%(uuid)0%dd' % len(match.group(0))) + return FileBasedSink._template_replace_window(shard_name_template) @staticmethod def _template_replace_num_shards(shard_name_template): @@ -391,7 +450,8 @@ def _template_replace_num_shards(shard_name_template): if match: shard_name_template = shard_name_template.replace( match.group(0), '%%(num_shards)0%dd' % len(match.group(0))) - return shard_name_template + #return shard_name_template + return FileBasedSink._template_replace_uuid(shard_name_template) @staticmethod def _template_to_format(shard_name_template): diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index 53215275e050..fb96a223a09f 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -35,6 +35,7 @@ import math import random import uuid +import apache_beam as beam from collections import namedtuple from typing import Any from typing import Iterator @@ -56,8 +57,10 @@ from apache_beam.transforms import core from apache_beam.transforms import ptransform from apache_beam.transforms import window +from apache_beam.transforms.core import DoFn from apache_beam.transforms.display import DisplayDataItem from apache_beam.transforms.display import HasDisplayData +from apache_beam.transforms.util import LogElements from apache_beam.utils import timestamp from apache_beam.utils import urns from apache_beam.utils.windowed_value import WindowedValue @@ -778,7 +781,7 @@ def open_writer(self, init_result, uid): """ raise NotImplementedError - def pre_finalize(self, init_result, writer_results): + def pre_finalize(self, init_result, writer_results, window=None): """Pre-finalization stage for sink. Called after all bundle writes are complete and before finalize_write. @@ -1127,47 +1130,155 @@ def __init__(self, sink: Sink) -> None: self.sink = sink def expand(self, pcoll): - do_once = pcoll.pipeline | 'DoOnce' >> core.Create([None]) - init_result_coll = do_once | 'InitializeWrite' >> core.Map( - lambda _, sink: sink.initialize_write(), self.sink) + if (pcoll.is_bounded): + do_once = pcoll.pipeline | 'DoOnce' >> core.Create([None]) + init_result_coll = do_once | 'InitializeWrite' >> core.Map( + lambda _, sink: sink.initialize_write(), self.sink) if getattr(self.sink, 'num_shards', 0): min_shards = self.sink.num_shards - if min_shards == 1: - keyed_pcoll = pcoll | core.Map(lambda x: (None, x)) - else: - keyed_pcoll = pcoll | core.ParDo(_RoundRobinKeyFn(), count=min_shards) - write_result_coll = ( + + if (pcoll.is_bounded): + if min_shards == 1: + keyed_pcoll = pcoll | core.Map(lambda x: (None, x)) + else: + keyed_pcoll = pcoll | core.ParDo(_RoundRobinKeyFn(), count=min_shards) + write_result_coll = ( + keyed_pcoll + | core.WindowInto(window.GlobalWindows()) + | core.GroupByKey() + | 'WriteBundles' >> core.ParDo( + _WriteKeyedBundleDoFn(self.sink), AsSingleton(init_result_coll))) + else: #unbounded PCollection needes to be written per window + if isinstance(pcoll.windowing.windowfn, window.GlobalWindows): + widowed_pcoll = ( + pcoll + | core.WindowInto(window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode.DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) + ) + else: #keep user windowing + widowed_pcoll = pcoll + if self.sink.convert_fn is not None: + widowed_pcoll = widowed_pcoll | core.ParDo(self.sink.convert_fn) + if min_shards == 1: + keyed_pcoll = widowed_pcoll | core.Map(lambda x: (None, x)) + else: + keyed_pcoll = widowed_pcoll | core.ParDo(_RoundRobinKeyFn(), count=min_shards) + init_result_window_coll = ( keyed_pcoll - | core.WindowInto(window.GlobalWindows()) - | core.GroupByKey() - | 'WriteBundles' >> core.ParDo( - _WriteKeyedBundleDoFn(self.sink), AsSingleton(init_result_coll))) + | 'Pair init' >> core.Map(lambda x: (None, x)) + | 'Pair init gbk' >> core.GroupByKey() + | 'InitializeWindowedWrite' >> core.Map( + lambda _, sink: sink.initialize_write(), self.sink) + #| 'LogElements init_result_window_coll' >> LogElements(prefix="init_result_window_coll :",with_window=True,with_timestamp=True,level=logging.INFO) + ) + + write_result_coll = ( + keyed_pcoll + | 'Group by random key' >> core.GroupByKey() + #| 'LogElements before WriteWindowedBundles' >> LogElements(prefix="before WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'WriteWindowedBundles' >> core.ParDo( + _WriteWindowedBundleDoFn(sink=self.sink,per_key=True), + AsSingleton(init_result_window_coll)) + #| 'LogElements' >> LogElements(prefix="after WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'Pair' >> core.Map(lambda x: (None, x)) + | core.GroupByKey() + | 'Extract' >> core.Map(lambda x: x[1]) + ) + pre_finalized_write_result_coll = ( + write_result_coll + | 'PreFinalize' >> core.ParDo( + _PreFinalizeWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) + ) + finalized_write_result_coll = ( + pre_finalized_write_result_coll + #| 'LogElements pre_finalized_write_result_coll' >> LogElements(prefix="pre_finalized_write_result_coll :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_window_coll), + AsSingleton(write_result_coll), + min_shards, + AsIter(pre_finalized_write_result_coll) + ).with_output_types(str) + ) + return finalized_write_result_coll else: + _LOGGER.info("*** WriteImpl min_shards undef so it's 1, and we write per Bundle") min_shards = 1 - write_result_coll = ( + if (pcoll.is_bounded): + write_result_coll = ( + pcoll + | core.WindowInto(window.GlobalWindows()) + | 'WriteBundles' >> core.ParDo( + _WriteBundleDoFn(self.sink), AsSingleton(init_result_coll)) + | 'Pair' >> core.Map(lambda x: (None, x)) + | core.GroupByKey() + | 'Extract' >> core.FlatMap(lambda x: x[1])) + else: #unbounded PCollection needes to be written per window + widowed_pcoll = ( pcoll - | core.WindowInto(window.GlobalWindows()) - | 'WriteBundles' >> core.ParDo( - _WriteBundleDoFn(self.sink), AsSingleton(init_result_coll)) - | 'Pair' >> core.Map(lambda x: (None, x)) - | core.GroupByKey() - | 'Extract' >> core.FlatMap(lambda x: x[1])) + | core.WindowInto(window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode.DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) + ) + init_result_window_coll = ( + widowed_pcoll + | 'Pair init' >> core.Map(lambda x: (None, x)) + | 'Pair init gbk' >> core.GroupByKey() + | 'InitializeWindowedWrite' >> core.Map( + lambda _, sink: sink.initialize_write(), self.sink) + ) + if self.sink.convert_fn is not None: + widowed_pcoll = widowed_pcoll | core.ParDo(self.sink.convert_fn) + write_result_coll = ( + widowed_pcoll + | 'WriteWindowedBundles' >> core.ParDo( + _WriteWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) + | 'LogElements' >> LogElements(prefix="after WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'Pair' >> core.Map(lambda x: (None, x)) + | core.GroupByKey() + | 'Extract' >> core.Map(lambda x: x[1]) + ) + pre_finalized_write_result_coll = ( + write_result_coll + | 'PreFinalize' >> core.ParDo( + _PreFinalizeWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) + ) + finalized_write_result_coll = ( + pre_finalized_write_result_coll + | 'LogElements 2' >> LogElements(prefix="before finalize :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_window_coll), + AsSingleton(write_result_coll), + min_shards, + AsIter(pre_finalized_write_result_coll) + ).with_output_types(str) + ) + return finalized_write_result_coll # PreFinalize should run before FinalizeWrite, and the two should not be # fused. - pre_finalize_coll = ( - do_once - | 'PreFinalize' >> core.FlatMap( - _pre_finalize, - self.sink, - AsSingleton(init_result_coll), - AsIter(write_result_coll))) - return do_once | 'FinalizeWrite' >> core.FlatMap( - _finalize_write, - self.sink, - AsSingleton(init_result_coll), - AsIter(write_result_coll), - min_shards, - AsSingleton(pre_finalize_coll)).with_output_types(str) + if (pcoll.is_bounded): + pre_finalize_coll = ( + do_once + | 'PreFinalize' >> core.FlatMap( + _pre_finalize, + self.sink, + AsSingleton(init_result_coll), + AsIter(write_result_coll))) + return (do_once | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_coll), + AsIter(write_result_coll), + min_shards, + AsSingleton(pre_finalize_coll)).with_output_types(str) + | 'LogElements after FinalizeWrite' >> LogElements(prefix='after FinalizeWrite ', with_window=False,level=logging.INFO) + ) class _WriteBundleDoFn(core.DoFn): @@ -1198,6 +1309,89 @@ def finish_bundle(self): self.writer.close(), window.GlobalWindow().max_timestamp(), [window.GlobalWindow()]) +class _PreFinalizeWindowedBundleDoFn(core.DoFn): + """A DoFn for writing elements to an iobase.Writer. + Opens a writer at the first element and closes the writer at finish_bundle(). + """ + def __init__( + self, + sink, + destination_fn=None, + temp_directory=None,): + self.sink = sink + self._temp_directory = temp_directory + self.destination_fn = destination_fn + + def display_data(self): + return {'sink_dd': self.sink} + + def process(self, + element, + init_result, + w=core.DoFn.WindowParam, + pane=core.DoFn.PaneInfoParam): + self.sink.pre_finalize(init_result=init_result,writer_results=element,window=w) + yield element + +class _WriteWindowedBundleDoFn(core.DoFn): + """A DoFn for writing elements to an iobase.Writer. + Opens a writer at the first element and closes the writer at finish_bundle(). + """ + def __init__( + self, + sink, + per_key=False): + self.sink = sink + self.per_key = per_key + + def display_data(self): + return {'sink_dd': self.sink} + + def start_bundle(self): + self.writer = {} + self.window = {} + self.init_result = {} + + def process(self, + element, + init_result, + w=core.DoFn.WindowParam, + pane=core.DoFn.PaneInfoParam): + + if self.per_key: + w_key = "%s_%s" % (w , element[0]) # key + else: + w_key = w + + if not w in self.writer : + # We ignore UUID collisions here since they are extremely rare. + self.window[w_key] = w + self.writer[w_key] = self.sink.open_writer(init_result, '%s_%s' % (w_key, uuid.uuid4())) + self.init_result[w_key] = init_result + #_LOGGER.info("*** _WriteWindowedBundleDoFn writer %s", self.writer[w_key].temp_shard_path) + if self.per_key: + for e in element[1]: # values + self.writer[w_key].write(e) # value + else: + self.writer[w_key].write(element) + if self.writer[w_key].at_capacity(): + yield self.writer[w_key].close() + self.writer[w_key] = None + + def finish_bundle(self): + for w_key, writer in self.writer.items(): + w = self.window[w_key] + if writer is not None: + closed = writer.temp_shard_path + try: + closed = writer.close() # TODO : improve sink closing for streaming + except ValueError as exp: + _LOGGER.info("*** _WriteWindowedBundleDoFn finish_bundle closed ERROR %s",exp) + yield WindowedValue( + closed, + timestamp=w.start, + windows=[w] # TODO(pabloem) HOW DO WE GET THE PANE + ) class _WriteKeyedBundleDoFn(core.DoFn): def __init__(self, sink): @@ -1224,7 +1418,8 @@ def _finalize_write( init_result, write_results, min_shards, - pre_finalize_results): + pre_finalize_results, + w=DoFn.WindowParam): write_results = list(write_results) extra_shards = [] if len(write_results) < min_shards: @@ -1235,10 +1430,20 @@ def _finalize_write( writer = sink.open_writer(init_result, str(uuid.uuid4())) extra_shards.append(writer.close()) outputs = sink.finalize_write( - init_result, write_results + extra_shards, pre_finalize_results) + init_result, write_results + extra_shards, pre_finalize_results, w) + outputs = list(outputs) + _LOGGER.info("*** _finalize_write outputs %s",outputs) if outputs: - return ( - window.TimestampedValue(v, timestamp.MAX_TIMESTAMP) for v in outputs) + if not isinstance(w, window.GlobalWindow): + #handle windowed finalize + yield ( + window.TimestampedValue(v, + timestamp=self.window.start, + windows=[w] + ) for v in outputs) + else: + return ( + window.TimestampedValue(v, timestamp.MAX_TIMESTAMP) for v in outputs) class _RoundRobinKeyFn(core.DoFn): diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 48c51428c17d..6c8baf15e7b3 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -48,6 +48,7 @@ from apache_beam.transforms import PTransform from apache_beam.transforms import window from apache_beam.typehints import schemas +from apache_beam.utils.windowed_value import WindowedValue try: import pyarrow as pa @@ -105,8 +106,13 @@ def __init__( self._buffer_size = record_batch_size self._record_batches = [] self._record_batches_byte_size = 0 + self._window = None - def process(self, row): + def process(self, + row, + w=DoFn.WindowParam, + pane=DoFn.PaneInfoParam): + self._window = w if len(self._buffer[0]) >= self._buffer_size: self._flush_buffer() @@ -123,7 +129,16 @@ def finish_bundle(self): self._flush_buffer() if self._record_batches_byte_size > 0: table = self._create_table() - yield window.GlobalWindows.windowed_value_at_end_of_window(table) + if self._window is None or isinstance(self._window, window.GlobalWindow): + # bounded input + yield window.GlobalWindows.windowed_value_at_end_of_window(table) + else: + # unbounded input + yield WindowedValue( + table, + timestamp=self._window.end, #or it could be max of timestamp of the rows processed + windows=[self._window] # TODO(pabloem) HOW DO WE GET THE PANE + ) def display_data(self): res = super().display_data() @@ -476,7 +491,9 @@ def __init__( file_name_suffix='', num_shards=0, shard_name_template=None, - mime_type='application/x-parquet'): + mime_type='application/x-parquet', + triggering_frequency=None, + ): """Initialize a WriteToParquet transform. Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of @@ -548,6 +565,8 @@ def __init__( is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A WriteToParquet transform usable for writing. @@ -567,10 +586,17 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) def expand(self, pcoll): + if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: + # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format + self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + if self._schema is None: try: beam_schema = schemas.schema_from_element_type(pcoll.element_type) @@ -583,7 +609,11 @@ def expand(self, pcoll): else: convert_fn = _RowDictionariesToArrowTable( self._schema, self._row_group_buffer_size, self._record_batch_size) - return pcoll | ParDo(convert_fn) | Write(self._sink) + if pcoll.is_bounded: + return pcoll | ParDo(convert_fn) | Write(self._sink) + else: + self._sink.convert_fn = convert_fn + return pcoll | Write(self._sink) def display_data(self): return { @@ -610,6 +640,7 @@ def __init__( num_shards=0, shard_name_template=None, mime_type='application/x-parquet', + triggering_frequency=None ): """Initialize a WriteToParquetBatched transform. @@ -668,11 +699,19 @@ def __init__( the shard number and shard count. When constructing a filename for a particular shard number, the upper-case letters 'S' and 'N' are replaced with the 0-padded shard number and shard count respectively. + W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().isoformat(), window.end.to_utc_datetime().isoformat()`` This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + set to 1 and only one file will be generated. + The default pattern used is '-SSSSS-of-NNNNN' if None is passed as the + shard_name_template and the PCollection is bounded. + The default pattern used is '-W-SSSSS-of-NNNNN' if None is passed as the + shard_name_template and the PCollection is unbounded. mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A WriteToParquetBatched transform usable for writing. @@ -688,10 +727,16 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) def expand(self, pcoll): + if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: + # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format + self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) return pcoll | Write(self._sink) def display_data(self): @@ -707,7 +752,8 @@ def _create_parquet_sink( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): return \ _ParquetSink( file_path_prefix, @@ -718,7 +764,8 @@ def _create_parquet_sink( file_name_suffix, num_shards, shard_name_template, - mime_type + mime_type, + triggering_frequency ) @@ -734,7 +781,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - mime_type): + mime_type, + triggering_frequency): super().__init__( file_path_prefix, file_name_suffix=file_name_suffix, @@ -744,7 +792,8 @@ def __init__( mime_type=mime_type, # Compression happens at the block level using the supplied codec, and # not at the file level. - compression_type=CompressionTypes.UNCOMPRESSED) + compression_type=CompressionTypes.UNCOMPRESSED, + triggering_frequency=triggering_frequency) self._schema = schema self._codec = codec if ARROW_MAJOR_VERSION == 1 and self._codec.lower() == "lz4": diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index e33ee4ec1129..5047fa5f1851 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -17,8 +17,11 @@ # pytype: skip-file import json +import glob import logging import os +import pytz +import re import shutil import tempfile import unittest @@ -26,6 +29,7 @@ import hamcrest as hc import pandas +import pyarrow import pytest from parameterized import param from parameterized import parameterized @@ -45,10 +49,13 @@ from apache_beam.io.parquetio import _create_parquet_sink from apache_beam.io.parquetio import _create_parquet_source from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher +from apache_beam.transforms.util import LogElements +from datetime import datetime try: import pyarrow as pa @@ -651,6 +658,149 @@ def test_read_all_from_parquet_with_filename(self): | ReadAllFromParquet(with_filename=True), equal_to(result)) +class GenerateEvent(beam.PTransform): + + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return (input + | TestStream() + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to_infinity()) + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #ParquetIO + pyschema = pyarrow.schema( + [('age', pyarrow.int64())] + ) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet" , + num_shards=num_shards, + schema=pyschema + ) + output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp_xyz/ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #ParquetIO + pyschema = pyarrow.schema( + [('age', pyarrow.int64())] + ) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet" , + shard_name_template=shard_name_template, + num_shards=num_shards, + schema=pyschema + ) + output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, + num_shards = 2, + shard_name_template = '-V-SSSSS-of-NNNNN', + triggering_frequency = 5): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #ParquetIO + pyschema = pyarrow.schema( + [('age', pyarrow.int64())] + ) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + schema=pyschema + ) + output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + #with 5s window size, the input should be processed by 5 windows with 2 shards per window + self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 0d7803bcabb1..90980860bac2 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -450,7 +450,8 @@ def __init__( *, max_records_per_shard=None, max_bytes_per_shard=None, - skip_if_empty=False): + skip_if_empty=False, + triggering_frequency=60): """Initialize a _TextSink. Args: @@ -493,6 +494,8 @@ def __init__( to exceed this value. This also tracks the uncompressed, not compressed, size of the shard. skip_if_empty: Don't write any shards if the PCollection is empty. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A _TextSink object usable for writing. @@ -507,7 +510,8 @@ def __init__( compression_type=compression_type, max_records_per_shard=max_records_per_shard, max_bytes_per_shard=max_bytes_per_shard, - skip_if_empty=skip_if_empty) + skip_if_empty=skip_if_empty, + triggering_frequency=triggering_frequency) self._append_trailing_newlines = append_trailing_newlines self._header = header self._footer = footer @@ -825,7 +829,8 @@ def __init__( *, max_records_per_shard=None, max_bytes_per_shard=None, - skip_if_empty=False): + skip_if_empty=False, + triggering_frequency=None): r"""Initialize a :class:`WriteToText` transform. Args: @@ -844,13 +849,18 @@ def __init__( the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template (str): A template string containing placeholders for - the shard number and shard count. Currently only ``''`` and - ``'-SSSSS-of-NNNNN'`` are patterns accepted by the service. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and ``'-V-SSSSS-of-NNNNN'`` + are patterns accepted by the service. When constructing a filename for a particular shard number, the upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded shard number and shard count respectively. This argument can be ``''`` in which case it behaves as if num_shards was set to 1 and only one file - will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'``. + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded PCollections. + W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().isoformat(), window.end.to_utc_datetime().isoformat()`` coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. compression_type (str): Used to handle compressed output files. Typical value is :class:`CompressionTypes.AUTO @@ -875,6 +885,8 @@ def __init__( skip_if_empty: Don't write any shards if the PCollection is empty. In case of an empty PCollection, this will still delete existing files having same file path and not create new ones. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. """ self._sink = _TextSink( @@ -889,9 +901,15 @@ def __init__( footer, max_records_per_shard=max_records_per_shard, max_bytes_per_shard=max_bytes_per_shard, - skip_if_empty=skip_if_empty) + skip_if_empty=skip_if_empty, + triggering_frequency=triggering_frequency) def expand(self, pcoll): + if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: + # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format + self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) return pcoll | Write(self._sink) diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index d1bfdf6bfd35..11759aac5247 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -23,6 +23,8 @@ import gzip import logging import os +import pytz +import re import shutil import tempfile import unittest @@ -44,11 +46,14 @@ from apache_beam.io.textio import WriteToText from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.test_utils import TempDir +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.core import Create +from apache_beam.transforms.util import LogElements from apache_beam.transforms.userstate import CombiningValueStateSpec from apache_beam.utils.timestamp import Timestamp +from datetime import datetime class DummyCoder(coders.Coder): @@ -1815,6 +1820,137 @@ def check_types(element): _ = pcoll | beam.Map(check_types) +class GenerateEvent(beam.PTransform): + + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return (input + | TestStream() + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to_infinity()) + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + num_shards=num_shards, + ) + output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp_xyz/ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + shard_name_template=shard_name_template, + num_shards=num_shards, + ) + output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, + num_shards = 2, + shard_name_template = '-V-SSSSS-of-NNNNN', + triggering_frequency = 5): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data() + ) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + ) + output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + #with 5s window size, the input should be processed by 5 windows with 2 shards per window + self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index da7ab0954339..e372cd241603 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -280,7 +280,8 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - compression_type): + compression_type, + triggering_frequency): """Initialize a TFRecordSink. See WriteToTFRecord for details.""" super().__init__( @@ -290,7 +291,8 @@ def __init__( num_shards=num_shards, shard_name_template=shard_name_template, mime_type='application/octet-stream', - compression_type=compression_type) + compression_type=compression_type, + triggering_frequency=triggering_frequency) def write_encoded_record(self, file_handle, value): _TFRecordUtil.write_record(file_handle, value) @@ -305,7 +307,8 @@ def __init__( file_name_suffix='', num_shards=0, shard_name_template=None, - compression_type=CompressionTypes.AUTO): + compression_type=CompressionTypes.AUTO, + triggering_frequency=None): """Initialize WriteToTFRecord transform. Args: @@ -326,6 +329,8 @@ def __init__( compression_type: Used to handle compressed output files. Typical value is CompressionTypes.AUTO, in which case the file_path's extension will be used to detect the compression. + triggering_frequency: (int) Every triggering_frequency duration, a window + will be triggered and all bundles in the window will be written. Returns: A WriteToTFRecord transform object. @@ -337,7 +342,14 @@ def __init__( file_name_suffix, num_shards, shard_name_template, - compression_type) + compression_type, + triggering_frequency) def expand(self, pcoll): + if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: + # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format + self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + return pcoll | Write(self._sink) diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index a867c0212ad3..0e48c8029dc6 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -21,11 +21,15 @@ import glob import gzip import io +import json import logging import os import pickle +import pytz import random import re +import shutil +import tempfile import unittest import zlib @@ -41,9 +45,12 @@ from apache_beam.io.tfrecordio import _TFRecordSink from apache_beam.io.tfrecordio import _TFRecordUtil from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.test_stream import TestStream from apache_beam.testing.test_utils import TempDir from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms.util import LogElements +from datetime import datetime try: import tensorflow.compat.v1 as tf # pylint: disable=import-error @@ -557,6 +564,143 @@ def test_end2end_read_write_read(self): actual_data = p | ReadFromTFRecord(path + '-*', validate=True) assert_that(actual_data, equal_to(expected_data)) +class GenerateEvent(beam.PTransform): + + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return (input + | TestStream() + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) + .advance_watermark_to_infinity()) + +class WriteStreamingTest(unittest.TestCase): + def setUp(self): + super().setUp() + self.tempdir = tempfile.mkdtemp() + + def tearDown(self): + if os.path.exists(self.tempdir): + shutil.rmtree(self.tempdir) + + def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + with TestPipeline() as p: + output = (p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) + ) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord" , + num_shards=num_shards, + ) + output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp_xyz/ouput_WriteToTFRecord-[1614556800.0, 1614556805.0)-00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + with TestPipeline() as p: + output = (p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) + ) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord" , + shard_name_template=shard_name_template, + num_shards=num_shards, + ) + output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, + num_shards = 2, + shard_name_template = '-V-SSSSS-of-NNNNN', + triggering_frequency = 5): + with TestPipeline() as p: + output = (p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) + ) + #TFrecordIO + output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", + file_name_suffix=".tfrecord", + shard_name_template=shard_name_template, + num_shards=num_shards, + triggering_frequency=triggering_frequency, + ) + output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord + # It captures: window_interval, shard_num, total_shards + pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): + match = pattern.match(file_name) + self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + #with 5s window size, the input should be processed by 5 windows with 2 shards per window + self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From eb3bb71d4a014f4f3c6704aa93acb23f96886829 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Tue, 29 Apr 2025 12:21:02 +0000 Subject: [PATCH 02/19] update changes.md : iobase patch for unbounded PColl and adding streaming writes for AvroIO, ParquetIO and TFRecordIO --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index b47ffc3da8a9..21d758ae7b64 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -64,6 +64,7 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* Support for streaming writes for AvroIO, ParquetIO and TFRecordIO ## New Features / Improvements From b5c6700336cf653c696fbf394b62aa55be1cdbb4 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Tue, 29 Apr 2025 15:16:30 +0000 Subject: [PATCH 03/19] apply yapf --- .../unbounded_sinks/generate_event.py | 149 ++++++++--- .../examples/unbounded_sinks/test_write.py | 168 ++++++------ .../unbounded_sinks/test_write_bounded.py | 55 ++-- sdks/python/apache_beam/io/avroio.py | 24 +- sdks/python/apache_beam/io/avroio_test.py | 239 +++++++++++------ sdks/python/apache_beam/io/iobase.py | 228 +++++++++------- sdks/python/apache_beam/io/parquetio.py | 43 +-- sdks/python/apache_beam/io/parquetio_test.py | 238 +++++++++++------ sdks/python/apache_beam/io/textio.py | 10 +- sdks/python/apache_beam/io/textio_test.py | 222 +++++++++++----- sdks/python/apache_beam/io/tfrecordio.py | 12 +- sdks/python/apache_beam/io/tfrecordio_test.py | 245 +++++++++++++----- 12 files changed, 1081 insertions(+), 552 deletions(-) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py index 7a429c1460b8..4065ce9a084e 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py @@ -23,43 +23,114 @@ class GenerateEvent(beam.PTransform): - @staticmethod - def sample_data(): - return GenerateEvent() + @staticmethod + def sample_data(): + return GenerateEvent() - def expand(self, input): - elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - my_schema = pyarrow.schema( - [pyarrow.field('age', pyarrow.int64())], - metadata = {"age":"the age int64"} - ) - #elem = [pyarrow.Table.from_pylist(elemlist, schema=my_schema)] - elem = elemlist - return (input - | TestStream() - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to_infinity()) + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + my_schema = pyarrow.schema([pyarrow.field('age', pyarrow.int64())], + metadata={"age": "the age int64"}) + #elem = [pyarrow.Table.from_pylist(elemlist, schema=my_schema)] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py index 08e110815a20..39ee6ddbb7ed 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -35,14 +35,20 @@ from apache_beam.options.pipeline_options import SetupOptions from apache_beam.runners.runner import PipelineResult + class CountEvents(beam.PTransform): + def expand(self, events): - return (events - | beam.WindowInto(FixedWindows(5), - trigger=AfterWatermark(), - accumulation_mode=AccumulationMode.DISCARDING, - allowed_lateness=Duration(seconds=0)) - | beam.CombineGlobally(beam.combiners.CountCombineFn()).without_defaults()) + return ( + events + | beam.WindowInto( + FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | beam.CombineGlobally( + beam.combiners.CountCombineFn()).without_defaults()) + def run(argv=None, save_main_session=True) -> PipelineResult: """Main entry point; defines and runs the wordcount pipeline.""" @@ -56,80 +62,82 @@ def run(argv=None, save_main_session=True) -> PipelineResult: p = beam.Pipeline(options=pipeline_options) - output = (p | GenerateEvent.sample_data() - #| 'Count' >> CountEvents() - #| 'Serialize' >> beam.Map(json.dumps) - #| beam.LogElements(prefix='before write ', with_window=False, with_timestamp=True,level=logging.INFO) - ) + output = ( + p | GenerateEvent.sample_data() + #| 'Count' >> CountEvents() + #| 'Serialize' >> beam.Map(json.dumps) + #| beam.LogElements(prefix='before write ', with_window=False, with_timestamp=True,level=logging.INFO) + ) #TextIO output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( file_path_prefix="__output__/ouput_WriteToText", file_name_suffix=".txt", #shard_name_template='-V-SSSSS-of-NNNNN', - num_shards=2, - #triggering_frequency=5, - ) - output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) - - #FileIO - output5 = ( output - | 'FileIO window' >> beam.WindowInto(FixedWindows(5), - trigger=AfterWatermark(), - accumulation_mode=AccumulationMode.DISCARDING, - allowed_lateness=Duration(seconds=0)) - | 'Serialize' >> beam.Map(json.dumps) - | 'FileIO WriteToFiles' >> WriteToFiles(path="__output__/output_WriteToFiles") + num_shards=2, #triggering_frequency=5, ) - - + output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) + + #FileIO + output5 = ( + output + | 'FileIO window' >> beam.WindowInto( + FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | 'Serialize' >> beam.Map(json.dumps) + | 'FileIO WriteToFiles' >> + WriteToFiles(path="__output__/output_WriteToFiles")) + #ParquetIO - pyschema = pyarrow.schema( - [('age', pyarrow.int64())] - ) - + pyschema = pyarrow.schema([('age', pyarrow.int64())]) + output4a = output | 'WriteToParquet' >> beam.io.WriteToParquet( - file_path_prefix="__output__/output_parquet", - #shard_name_template='-V-SSSSS-of-NNNNN', - file_name_suffix=".parquet" , - num_shards=2, - triggering_frequency=5, - schema=pyschema - ) - output4a | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet 4a ', with_window=True,level=logging.INFO) - - output4aw = ( output - | 'ParquetIO window' >> beam.WindowInto(FixedWindows(20), + file_path_prefix="__output__/output_parquet", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".parquet", + num_shards=2, + triggering_frequency=5, + schema=pyschema) + output4a | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet 4a ', with_window=True, level=logging.INFO) + + output4aw = ( + output + | 'ParquetIO window' >> beam.WindowInto( + FixedWindows(20), trigger=AfterWatermark(), accumulation_mode=AccumulationMode.DISCARDING, allowed_lateness=Duration(seconds=0)) | 'WriteToParquet windowed' >> beam.io.WriteToParquet( file_path_prefix="__output__/output_parquet", shard_name_template='-W-SSSSS-of-NNNNN', - file_name_suffix=".parquet" , + file_name_suffix=".parquet", num_shards=2, - schema=pyschema - ) - ) - output4aw | 'LogElements after WriteToParquet windowed' >> LogElements(prefix='after WriteToParquet 4aw ', with_window=True,level=logging.INFO) - + schema=pyschema)) + output4aw | 'LogElements after WriteToParquet windowed' >> LogElements( + prefix='after WriteToParquet 4aw ', with_window=True, level=logging.INFO) - output4b = (output - | 'To PyArrow Table' >> beam.Map(lambda x: pyarrow.Table.from_pylist([x], schema=pyschema)) - | 'WriteToParquetBatched to parquet' >> beam.io.WriteToParquetBatched( + output4b = ( + output + | 'To PyArrow Table' >> + beam.Map(lambda x: pyarrow.Table.from_pylist([x], schema=pyschema)) + | 'WriteToParquetBatched to parquet' >> beam.io.WriteToParquetBatched( file_path_prefix="__output__/output_parquet_batched", shard_name_template='-V-SSSSS-of-NNNNN', - file_name_suffix=".parquet" , + file_name_suffix=".parquet", num_shards=2, triggering_frequency=5, - schema=pyschema - ) - ) - output4b | 'LogElements after WriteToParquetBatched' >> LogElements(prefix='after WriteToParquetBatched 4b ', with_window=True,level=logging.INFO) - + schema=pyschema)) + output4b | 'LogElements after WriteToParquetBatched' >> LogElements( + prefix='after WriteToParquetBatched 4b ', + with_window=True, + level=logging.INFO) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description + #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time 'fields': [ # this defines actual keys & their types @@ -137,34 +145,34 @@ def run(argv=None, save_main_session=True) -> PipelineResult: ], } output5 = output | 'WriteToAvro' >> beam.io.WriteToAvro( - file_path_prefix="__output__/output_avro", - #shard_name_template='-V-SSSSS-of-NNNNN', - file_name_suffix=".avro" , - num_shards=2, - #triggering_frequency=5, - schema=avroschema - ) - output5 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro 5 ', with_window=True,level=logging.INFO) - + file_path_prefix="__output__/output_avro", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".avro", + num_shards=2, + #triggering_frequency=5, + schema=avroschema) + output5 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro 5 ', with_window=True, level=logging.INFO) + #TFrecordIO - output6 = (output - | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) - | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( - file_path_prefix="__output__/output_tfrecord", - #shard_name_template='-V-SSSSS-of-NNNNN', - file_name_suffix=".tfrecord" , - num_shards=2, - triggering_frequency=5 - ) - ) - output6 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord 6 ', with_window=True,level=logging.INFO) - - + output6 = ( + output + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) + | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( + file_path_prefix="__output__/output_tfrecord", + #shard_name_template='-V-SSSSS-of-NNNNN', + file_name_suffix=".tfrecord", + num_shards=2, + triggering_frequency=5)) + output6 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord 6 ', with_window=True, level=logging.INFO) + # Execute the pipeline and return the result. result = p.run() result.wait_until_finish() return result + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) - run() \ No newline at end of file + run() diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py index 358ebd4dd210..712364c61a99 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py @@ -36,14 +36,20 @@ from apache_beam.options.pipeline_options import SetupOptions from apache_beam.runners.runner import PipelineResult + class CountEvents(beam.PTransform): + def expand(self, events): - return (events - | beam.WindowInto(FixedWindows(5), - trigger=AfterWatermark(), - accumulation_mode=AccumulationMode.DISCARDING, - allowed_lateness=Duration(seconds=0)) - | beam.CombineGlobally(beam.combiners.CountCombineFn()).without_defaults()) + return ( + events + | beam.WindowInto( + FixedWindows(5), + trigger=AfterWatermark(), + accumulation_mode=AccumulationMode.DISCARDING, + allowed_lateness=Duration(seconds=0)) + | beam.CombineGlobally( + beam.combiners.CountCombineFn()).without_defaults()) + def run(argv=None, save_main_session=True) -> PipelineResult: """Main entry point; defines and runs the wordcount pipeline.""" @@ -57,28 +63,40 @@ def run(argv=None, save_main_session=True) -> PipelineResult: p = beam.Pipeline(options=pipeline_options) - output = (p | beam.Create([{'age': 10}, {'age': 20}, {'age': 30}]) - #| beam.CombineGlobally(AverageFn()) - #| 'Serialize' >> beam.Map(json.dumps) - | beam.LogElements(prefix='before write ', with_window=False,level=logging.INFO) ) + output = ( + p | beam.Create([{ + 'age': 10 + }, { + 'age': 20 + }, { + 'age': 30 + }]) + #| beam.CombineGlobally(AverageFn()) + #| 'Serialize' >> beam.Map(json.dumps) + | beam.LogElements( + prefix='before write ', with_window=False, level=logging.INFO)) #OK in batch - output2 = output | 'Write to text' >> WriteToText(file_path_prefix="__output_batch__/ouput_WriteToText",file_name_suffix=".txt",shard_name_template='-U-SSSSS-of-NNNNN') - output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=False,level=logging.INFO) + output2 = output | 'Write to text' >> WriteToText( + file_path_prefix="__output_batch__/ouput_WriteToText", + file_name_suffix=".txt", + shard_name_template='-U-SSSSS-of-NNNNN') + output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=False, level=logging.INFO) #OK in batch and stream # output3 = (output | 'Serialize' >> beam.Map(json.dumps) - # | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") + # | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") # ) - # output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) + # output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) #KO - ValueError: GroupByKey cannot be applied to an unbounded PCollection with global windowing and a default trigger # output4 = output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_parquet", # schema= # pyarrow.schema( # [('age', pyarrow.int64())] - # ) + # ) # ) - # output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) + # output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) # output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="output", # schema= # pyarrow.schema( @@ -87,14 +105,13 @@ def run(argv=None, save_main_session=True) -> PipelineResult: # record_batch_size = 10, # num_shards=0 # ) - - # Execute the pipeline and return the result. result = p.run() result.wait_until_finish() return result + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) - run() \ No newline at end of file + run() diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 7ef0f9bf9766..1b342d339355 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -86,6 +86,7 @@ class ReadFromAvro(PTransform): that comply with the schema contained in the Avro file that contains those records. """ + def __init__( self, file_pattern=None, @@ -288,6 +289,7 @@ def expand(self, pbegin): class _AvroUtils(object): + @staticmethod def advance_file_past_next_sync_marker(f, sync_marker): buf_size = 10000 @@ -323,6 +325,7 @@ class _FastAvroSource(filebasedsource.FileBasedSource): TODO: remove ``_AvroSource`` in favor of using ``_FastAvroSource`` everywhere once it has been more widely tested """ + def read_records(self, file_name, range_tracker): next_block_start = -1 @@ -368,6 +371,7 @@ class WriteToAvro(beam.transforms.PTransform): If the input has a schema, a corresponding avro schema will be automatically generated and used to write the output records.""" + def __init__( self, file_path_prefix, @@ -415,14 +419,8 @@ def __init__( """ self._schema = schema self._sink_provider = lambda avro_schema: _create_avro_sink( - file_path_prefix, - avro_schema, - codec, - file_name_suffix, - num_shards, - shard_name_template, - mime_type, - triggering_frequency) + file_path_prefix, avro_schema, codec, file_name_suffix, num_shards, + shard_name_template, mime_type, triggering_frequency) def expand(self, pcoll): if self._schema: @@ -442,9 +440,11 @@ def expand(self, pcoll): if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE - self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) - self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) - + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) + return records | beam.io.iobase.Write(self._sink) def display_data(self): @@ -478,6 +478,7 @@ def _create_avro_sink( class _BaseAvroSink(filebasedsink.FileBasedSink): """A base for a sink for avro files. """ + def __init__( self, file_path_prefix, @@ -511,6 +512,7 @@ def display_data(self): class _FastAvroSink(_BaseAvroSink): """A sink for avro files using FastAvro. """ + def __init__( self, file_path_prefix, diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index dd71bb22495e..2968c51c08fd 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -179,8 +179,8 @@ def test_schema_read_write(self): @pytest.mark.xlang_sql_expansion_service @unittest.skipIf( - TestPipeline().get_pipeline_options().view_as(StandardOptions).runner is - None, + TestPipeline().get_pipeline_options().view_as(StandardOptions).runner + is None, "Must be run with a runner that supports staging java artifacts.") def test_avro_schema_to_beam_schema_with_nullable_atomic_fields(self): records = [] @@ -381,6 +381,7 @@ def test_read_with_splitting_pattern(self): self._run_avro_test(pattern, 100, True, expected_result) def test_dynamic_work_rebalancing_exhaustive(self): + def compare_split_points(file_name): source = _FastAvroSource(file_name) splits = [ @@ -610,6 +611,7 @@ def test_writer_open_and_close(self): class TestFastAvro(AvroBase, unittest.TestCase): + def __init__(self, methodName='runTest'): super().__init__(methodName) self.SCHEMA = parse_schema(json.loads(self.SCHEMA_STRING)) @@ -631,45 +633,121 @@ def _write_data( self._temp_files.append(f.name) return f.name + class GenerateEvent(beam.PTransform): - @staticmethod - def sample_data(): - return GenerateEvent() - - def expand(self, input): - elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - elem = elemlist - return (input - | TestStream() - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to_infinity()) + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + class WriteStreamingTest(unittest.TestCase): + def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -678,13 +756,13 @@ def tearDown(self): if os.path.exists(self.tempdir): shutil.rmtree(self.tempdir) - def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description + #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time 'fields': [ # this defines actual keys & their types @@ -692,34 +770,38 @@ def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = ], } output2 = output | 'WriteToAvro' >> beam.io.WriteToAvro( - file_path_prefix=self.tempdir + "/ouput_WriteToAvro", - file_name_suffix=".avro" , - num_shards=num_shards, - schema=avroschema - ) - output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + file_path_prefix=self.tempdir + "/ouput_WriteToAvro", + file_name_suffix=".avro", + num_shards=num_shards, + schema=avroschema) + output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp_xyz/ouput_WriteToAvro-[1614556800.0, 1614556805.0)-00000-of-00002.avro - # It captures: window_interval, shard_num, total_shards + # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) - - def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description + #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time 'fields': [ # this defines actual keys & their types @@ -731,34 +813,38 @@ def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2 file_name_suffix=".avro", shard_name_template=shard_name_template, num_shards=num_shards, - schema=avroschema - ) - output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + schema=avroschema) + output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) - def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, - num_shards = 2, - shard_name_template = '-V-SSSSS-of-NNNNN', - triggering_frequency = 5): + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description + #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time 'fields': [ # this defines actual keys & their types @@ -771,24 +857,29 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, shard_name_template=shard_name_template, num_shards=num_shards, triggering_frequency=triggering_frequency, - schema=avroschema - ) - output2 | 'LogElements after WriteToAvro' >> LogElements(prefix='after WriteToAvro ', with_window=True,level=logging.INFO) + schema=avroschema) + output2 | 'LogElements after WriteToAvro' >> LogElements( + prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) #with 5s window size, the input should be processed by 5 windows with 2 shards per window - self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index fb96a223a09f..abc58f2031f6 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -146,6 +146,7 @@ class BoundedSource(SourceBase): implementations may invoke methods of ``BoundedSource`` objects through multi-threaded and/or reentrant execution modes. """ + def estimate_size(self) -> Optional[int]: """Estimates the size of source in bytes. @@ -849,6 +850,7 @@ class Writer(object): See ``iobase.Sink`` for more detailed documentation about the process of writing to a sink. """ + def write(self, value): """Writes a value to the sink using the current writer. """ @@ -1125,6 +1127,7 @@ def from_runner_api_parameter( class WriteImpl(ptransform.PTransform): """Implements the writing of custom sinks.""" + def __init__(self, sink: Sink) -> None: super().__init__() self.sink = sink @@ -1136,7 +1139,7 @@ def expand(self, pcoll): lambda _, sink: sink.initialize_write(), self.sink) if getattr(self.sink, 'num_shards', 0): min_shards = self.sink.num_shards - + if (pcoll.is_bounded): if min_shards == 1: keyed_pcoll = pcoll | core.Map(lambda x: (None, x)) @@ -1147,65 +1150,66 @@ def expand(self, pcoll): | core.WindowInto(window.GlobalWindows()) | core.GroupByKey() | 'WriteBundles' >> core.ParDo( - _WriteKeyedBundleDoFn(self.sink), AsSingleton(init_result_coll))) - else: #unbounded PCollection needes to be written per window + _WriteKeyedBundleDoFn(self.sink), AsSingleton(init_result_coll)) + ) + else: #unbounded PCollection needes to be written per window if isinstance(pcoll.windowing.windowfn, window.GlobalWindows): widowed_pcoll = ( - pcoll - | core.WindowInto(window.FixedWindows(self.sink.triggering_frequency), - trigger=beam.transforms.trigger.AfterWatermark(), - accumulation_mode=beam.transforms.trigger.AccumulationMode.DISCARDING, - allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) - ) - else: #keep user windowing + pcoll + | core.WindowInto( + window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + else: #keep user windowing widowed_pcoll = pcoll if self.sink.convert_fn is not None: widowed_pcoll = widowed_pcoll | core.ParDo(self.sink.convert_fn) if min_shards == 1: keyed_pcoll = widowed_pcoll | core.Map(lambda x: (None, x)) else: - keyed_pcoll = widowed_pcoll | core.ParDo(_RoundRobinKeyFn(), count=min_shards) + keyed_pcoll = widowed_pcoll | core.ParDo( + _RoundRobinKeyFn(), count=min_shards) init_result_window_coll = ( - keyed_pcoll + keyed_pcoll | 'Pair init' >> core.Map(lambda x: (None, x)) | 'Pair init gbk' >> core.GroupByKey() | 'InitializeWindowedWrite' >> core.Map( - lambda _, sink: sink.initialize_write(), self.sink) + lambda _, sink: sink.initialize_write(), self.sink) #| 'LogElements init_result_window_coll' >> LogElements(prefix="init_result_window_coll :",with_window=True,with_timestamp=True,level=logging.INFO) ) - + write_result_coll = ( - keyed_pcoll + keyed_pcoll | 'Group by random key' >> core.GroupByKey() #| 'LogElements before WriteWindowedBundles' >> LogElements(prefix="before WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) | 'WriteWindowedBundles' >> core.ParDo( - _WriteWindowedBundleDoFn(sink=self.sink,per_key=True), + _WriteWindowedBundleDoFn(sink=self.sink, per_key=True), AsSingleton(init_result_window_coll)) #| 'LogElements' >> LogElements(prefix="after WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) | 'Pair' >> core.Map(lambda x: (None, x)) | core.GroupByKey() - | 'Extract' >> core.Map(lambda x: x[1]) - ) + | 'Extract' >> core.Map(lambda x: x[1])) pre_finalized_write_result_coll = ( - write_result_coll - | 'PreFinalize' >> core.ParDo( - _PreFinalizeWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) - ) + write_result_coll + | 'PreFinalize' >> core.ParDo( + _PreFinalizeWindowedBundleDoFn(self.sink), + AsSingleton(init_result_window_coll))) finalized_write_result_coll = ( - pre_finalized_write_result_coll - #| 'LogElements pre_finalized_write_result_coll' >> LogElements(prefix="pre_finalized_write_result_coll :",with_window=True,with_timestamp=True,level=logging.INFO) - | 'FinalizeWrite' >> core.FlatMap( - _finalize_write, - self.sink, - AsSingleton(init_result_window_coll), - AsSingleton(write_result_coll), - min_shards, - AsIter(pre_finalized_write_result_coll) - ).with_output_types(str) - ) + pre_finalized_write_result_coll + #| 'LogElements pre_finalized_write_result_coll' >> LogElements(prefix="pre_finalized_write_result_coll :",with_window=True,with_timestamp=True,level=logging.INFO) + | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_window_coll), + AsSingleton(write_result_coll), + min_shards, + AsIter(pre_finalized_write_result_coll)).with_output_types(str)) return finalized_write_result_coll else: - _LOGGER.info("*** WriteImpl min_shards undef so it's 1, and we write per Bundle") + _LOGGER.info( + "*** WriteImpl min_shards undef so it's 1, and we write per Bundle") min_shards = 1 if (pcoll.is_bounded): write_result_coll = ( @@ -1216,49 +1220,55 @@ def expand(self, pcoll): | 'Pair' >> core.Map(lambda x: (None, x)) | core.GroupByKey() | 'Extract' >> core.FlatMap(lambda x: x[1])) - else: #unbounded PCollection needes to be written per window + else: #unbounded PCollection needes to be written per window widowed_pcoll = ( - pcoll - | core.WindowInto(window.FixedWindows(self.sink.triggering_frequency), - trigger=beam.transforms.trigger.AfterWatermark(), - accumulation_mode=beam.transforms.trigger.AccumulationMode.DISCARDING, - allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) - ) + pcoll + | core.WindowInto( + window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) init_result_window_coll = ( - widowed_pcoll + widowed_pcoll | 'Pair init' >> core.Map(lambda x: (None, x)) | 'Pair init gbk' >> core.GroupByKey() | 'InitializeWindowedWrite' >> core.Map( - lambda _, sink: sink.initialize_write(), self.sink) - ) + lambda _, sink: sink.initialize_write(), self.sink)) if self.sink.convert_fn is not None: widowed_pcoll = widowed_pcoll | core.ParDo(self.sink.convert_fn) write_result_coll = ( - widowed_pcoll + widowed_pcoll | 'WriteWindowedBundles' >> core.ParDo( - _WriteWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) - | 'LogElements' >> LogElements(prefix="after WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) + _WriteWindowedBundleDoFn(self.sink), + AsSingleton(init_result_window_coll)) + | 'LogElements' >> LogElements( + prefix="after WriteWindowedBundles :", + with_window=True, + with_timestamp=True, + level=logging.INFO) | 'Pair' >> core.Map(lambda x: (None, x)) | core.GroupByKey() - | 'Extract' >> core.Map(lambda x: x[1]) - ) + | 'Extract' >> core.Map(lambda x: x[1])) pre_finalized_write_result_coll = ( - write_result_coll - | 'PreFinalize' >> core.ParDo( - _PreFinalizeWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) - ) + write_result_coll + | 'PreFinalize' >> core.ParDo( + _PreFinalizeWindowedBundleDoFn(self.sink), + AsSingleton(init_result_window_coll))) finalized_write_result_coll = ( - pre_finalized_write_result_coll - | 'LogElements 2' >> LogElements(prefix="before finalize :",with_window=True,with_timestamp=True,level=logging.INFO) - | 'FinalizeWrite' >> core.FlatMap( - _finalize_write, - self.sink, - AsSingleton(init_result_window_coll), - AsSingleton(write_result_coll), - min_shards, - AsIter(pre_finalized_write_result_coll) - ).with_output_types(str) - ) + pre_finalized_write_result_coll + | 'LogElements 2' >> LogElements( + prefix="before finalize :", + with_window=True, + with_timestamp=True, + level=logging.INFO) + | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_window_coll), + AsSingleton(write_result_coll), + min_shards, + AsIter(pre_finalized_write_result_coll)).with_output_types(str)) return finalized_write_result_coll # PreFinalize should run before FinalizeWrite, and the two should not be # fused. @@ -1270,21 +1280,25 @@ def expand(self, pcoll): self.sink, AsSingleton(init_result_coll), AsIter(write_result_coll))) - return (do_once | 'FinalizeWrite' >> core.FlatMap( - _finalize_write, - self.sink, - AsSingleton(init_result_coll), - AsIter(write_result_coll), - min_shards, - AsSingleton(pre_finalize_coll)).with_output_types(str) - | 'LogElements after FinalizeWrite' >> LogElements(prefix='after FinalizeWrite ', with_window=False,level=logging.INFO) - ) + return ( + do_once | 'FinalizeWrite' >> core.FlatMap( + _finalize_write, + self.sink, + AsSingleton(init_result_coll), + AsIter(write_result_coll), + min_shards, + AsSingleton(pre_finalize_coll)).with_output_types(str) + | 'LogElements after FinalizeWrite' >> LogElements( + prefix='after FinalizeWrite ', + with_window=False, + level=logging.INFO)) class _WriteBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ + def __init__(self, sink): self.sink = sink @@ -1309,15 +1323,18 @@ def finish_bundle(self): self.writer.close(), window.GlobalWindow().max_timestamp(), [window.GlobalWindow()]) + class _PreFinalizeWindowedBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ + def __init__( self, sink, destination_fn=None, - temp_directory=None,): + temp_directory=None, + ): self.sink = sink self._temp_directory = temp_directory self.destination_fn = destination_fn @@ -1325,22 +1342,23 @@ def __init__( def display_data(self): return {'sink_dd': self.sink} - def process(self, - element, - init_result, - w=core.DoFn.WindowParam, + def process( + self, + element, + init_result, + w=core.DoFn.WindowParam, pane=core.DoFn.PaneInfoParam): - self.sink.pre_finalize(init_result=init_result,writer_results=element,window=w) + self.sink.pre_finalize( + init_result=init_result, writer_results=element, window=w) yield element + class _WriteWindowedBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ - def __init__( - self, - sink, - per_key=False): + + def __init__(self, sink, per_key=False): self.sink = sink self.per_key = per_key @@ -1352,26 +1370,28 @@ def start_bundle(self): self.window = {} self.init_result = {} - def process(self, - element, - init_result, - w=core.DoFn.WindowParam, + def process( + self, + element, + init_result, + w=core.DoFn.WindowParam, pane=core.DoFn.PaneInfoParam): if self.per_key: - w_key = "%s_%s" % (w , element[0]) # key + w_key = "%s_%s" % (w, element[0]) # key else: w_key = w - if not w in self.writer : + if not w in self.writer: # We ignore UUID collisions here since they are extremely rare. self.window[w_key] = w - self.writer[w_key] = self.sink.open_writer(init_result, '%s_%s' % (w_key, uuid.uuid4())) + self.writer[w_key] = self.sink.open_writer( + init_result, '%s_%s' % (w_key, uuid.uuid4())) self.init_result[w_key] = init_result #_LOGGER.info("*** _WriteWindowedBundleDoFn writer %s", self.writer[w_key].temp_shard_path) if self.per_key: for e in element[1]: # values - self.writer[w_key].write(e) # value + self.writer[w_key].write(e) # value else: self.writer[w_key].write(element) if self.writer[w_key].at_capacity(): @@ -1384,16 +1404,19 @@ def finish_bundle(self): if writer is not None: closed = writer.temp_shard_path try: - closed = writer.close() # TODO : improve sink closing for streaming + closed = writer.close() # TODO : improve sink closing for streaming except ValueError as exp: - _LOGGER.info("*** _WriteWindowedBundleDoFn finish_bundle closed ERROR %s",exp) + _LOGGER.info( + "*** _WriteWindowedBundleDoFn finish_bundle closed ERROR %s", exp) yield WindowedValue( closed, timestamp=w.start, windows=[w] # TODO(pabloem) HOW DO WE GET THE PANE ) + class _WriteKeyedBundleDoFn(core.DoFn): + def __init__(self, sink): self.sink = sink @@ -1432,21 +1455,20 @@ def _finalize_write( outputs = sink.finalize_write( init_result, write_results + extra_shards, pre_finalize_results, w) outputs = list(outputs) - _LOGGER.info("*** _finalize_write outputs %s",outputs) + #_LOGGER.info("*** _finalize_write outputs %s",outputs) if outputs: if not isinstance(w, window.GlobalWindow): #handle windowed finalize yield ( - window.TimestampedValue(v, - timestamp=self.window.start, - windows=[w] - ) for v in outputs) + window.TimestampedValue(v, timestamp=self.window.start, windows=[w]) + for v in outputs) else: return ( window.TimestampedValue(v, timestamp.MAX_TIMESTAMP) for v in outputs) class _RoundRobinKeyFn(core.DoFn): + def start_bundle(self): self.counter = None @@ -1470,6 +1492,7 @@ class RestrictionTracker(object): * https://s.apache.org/splittable-do-fn * https://s.apache.org/splittable-do-fn-python-sdk """ + def current_restriction(self): """Returns the current restriction. @@ -1604,6 +1627,7 @@ class WatermarkEstimator(object): Internal state must not be updated asynchronously. """ + def get_estimator_state(self): """Get current state of the WatermarkEstimator instance, which can be used to recreate the WatermarkEstimator when processing the restriction. See @@ -1629,6 +1653,7 @@ def observe_timestamp(self, timestamp: timestamp.Timestamp) -> None: class RestrictionProgress(object): """Used to record the progress of a restriction.""" + def __init__(self, **kwargs): # Only accept keyword arguments. self._fraction = kwargs.pop('fraction', None) @@ -1683,6 +1708,7 @@ def with_completed(self, completed: int) -> 'RestrictionProgress': class _SDFBoundedSourceRestriction(object): """ A restriction wraps SourceBundle and RangeTracker. """ + def __init__(self, source_bundle, range_tracker=None): self._source_bundle = source_bundle self._range_tracker = range_tracker @@ -1746,6 +1772,7 @@ class _SDFBoundedSourceRestrictionTracker(RestrictionTracker): Delegated RangeTracker guarantees synchronization safety. """ + def __init__(self, restriction): if not isinstance(restriction, _SDFBoundedSourceRestriction): raise ValueError( @@ -1782,6 +1809,7 @@ def is_bounded(self): class _SDFBoundedSourceWrapperRestrictionCoder(coders.Coder): + def decode(self, value): return _SDFBoundedSourceRestriction(SourceBundle(*pickler.loads(value))) @@ -1800,6 +1828,7 @@ class _SDFBoundedSourceRestrictionProvider(core.RestrictionProvider): This restriction provider initializes restriction based on input element that is expected to be of BoundedSource type. """ + def __init__(self, desired_chunk_size=None, restriction_coder=None): self._desired_chunk_size = desired_chunk_size self._restriction_coder = ( @@ -1849,12 +1878,15 @@ class SDFBoundedSourceReader(PTransform): NOTE: This transform can only be used with beam_fn_api enabled. """ + def __init__(self, data_to_display=None): self._data_to_display = data_to_display or {} super().__init__() def _create_sdf_bounded_source_dofn(self): + class SDFBoundedSourceDoFn(core.DoFn): + def __init__(self, dd): self._dd = dd diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 6c8baf15e7b3..8c339def9630 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -77,6 +77,7 @@ class _ArrowTableToRowDictionaries(DoFn): """ A DoFn that consumes an Arrow table and yields a python dictionary for each row in the table.""" + def process(self, table, with_filename=False): if with_filename: file_name = table[0] @@ -95,6 +96,7 @@ def process(self, table, with_filename=False): class _RowDictionariesToArrowTable(DoFn): """ A DoFn that consumes python dictionarys and yields a pyarrow table.""" + def __init__( self, schema, @@ -108,10 +110,7 @@ def __init__( self._record_batches_byte_size = 0 self._window = None - def process(self, - row, - w=DoFn.WindowParam, - pane=DoFn.PaneInfoParam): + def process(self, row, w=DoFn.WindowParam, pane=DoFn.PaneInfoParam): self._window = w if len(self._buffer[0]) >= self._buffer_size: self._flush_buffer() @@ -135,9 +134,10 @@ def finish_bundle(self): else: # unbounded input yield WindowedValue( - table, - timestamp=self._window.end, #or it could be max of timestamp of the rows processed - windows=[self._window] # TODO(pabloem) HOW DO WE GET THE PANE + table, + timestamp=self._window. + end, #or it could be max of timestamp of the rows processed + windows=[self._window] # TODO(pabloem) HOW DO WE GET THE PANE ) def display_data(self): @@ -169,6 +169,7 @@ def _flush_buffer(self): class _ArrowTableToBeamRows(DoFn): + def __init__(self, beam_type): self._beam_type = beam_type @@ -181,6 +182,7 @@ def infer_output_type(self, input_type): class _BeamRowsToArrowTable(DoFn): + @DoFn.yields_elements def process_batch(self, element: pa.Table) -> Iterator[pa.Table]: yield element @@ -190,6 +192,7 @@ class ReadFromParquetBatched(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is currently experimental. No backward-compatibility guarantees.""" + def __init__( self, file_pattern=None, min_bundle_size=0, validate=True, columns=None): """ Initializes :class:`~ReadFromParquetBatched` @@ -247,6 +250,7 @@ def display_data(self): class ReadFromParquet(PTransform): """A `PTransform` for reading Parquet files.""" + def __init__( self, file_pattern=None, @@ -385,6 +389,7 @@ def expand(self, pvalue): class ReadAllFromParquet(PTransform): + def __init__(self, with_filename=False, **kwargs): self._with_filename = with_filename self._read_batches = ReadAllFromParquetBatched( @@ -396,6 +401,7 @@ def expand(self, pvalue): class _ParquetUtils(object): + @staticmethod def find_first_row_group_index(pf, start_offset): for i in range(_ParquetUtils.get_number_of_row_groups(pf)): @@ -421,6 +427,7 @@ def get_number_of_row_groups(pf): class _ParquetSource(filebasedsource.FileBasedSource): """A source for reading Parquet files. """ + def __init__( self, file_pattern, min_bundle_size=0, validate=False, columns=None): super().__init__( @@ -479,6 +486,7 @@ def split_points_unclaimed(stop_position): class WriteToParquet(PTransform): """A ``PTransform`` for writing parquet files. """ + def __init__( self, file_path_prefix, @@ -493,7 +501,7 @@ def __init__( shard_name_template=None, mime_type='application/x-parquet', triggering_frequency=None, - ): + ): """Initialize a WriteToParquet transform. Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of @@ -594,9 +602,11 @@ def expand(self, pcoll): if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE - self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) - self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) - + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) + if self._schema is None: try: beam_schema = schemas.schema_from_element_type(pcoll.element_type) @@ -629,6 +639,7 @@ class WriteToParquetBatched(PTransform): This ``PTransform`` is currently experimental. No backward-compatibility guarantees. """ + def __init__( self, file_path_prefix, @@ -640,8 +651,7 @@ def __init__( num_shards=0, shard_name_template=None, mime_type='application/x-parquet', - triggering_frequency=None - ): + triggering_frequency=None): """Initialize a WriteToParquetBatched transform. Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of @@ -735,8 +745,10 @@ def expand(self, pcoll): if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE - self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) - self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) return pcoll | Write(self._sink) def display_data(self): @@ -771,6 +783,7 @@ def _create_parquet_sink( class _ParquetSink(filebasedsink.FileBasedSink): """A sink for parquet files from batches.""" + def __init__( self, file_path_prefix, diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index 5047fa5f1851..ff9311eea832 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -72,6 +72,7 @@ @unittest.skipIf(pa is None, "PyArrow is not installed.") @pytest.mark.uses_pyarrow class TestParquet(unittest.TestCase): + def setUp(self): # Reducing the size of thread pools. Without this test execution may fail in # environments with limited amount of resources. @@ -658,45 +659,121 @@ def test_read_all_from_parquet_with_filename(self): | ReadAllFromParquet(with_filename=True), equal_to(result)) + class GenerateEvent(beam.PTransform): - @staticmethod - def sample_data(): - return GenerateEvent() - - def expand(self, input): - elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - elem = elemlist - return (input - | TestStream() - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to_infinity()) + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + class WriteStreamingTest(unittest.TestCase): + def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -705,102 +782,109 @@ def tearDown(self): if os.path.exists(self.tempdir): shutil.rmtree(self.tempdir) - def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema( - [('age', pyarrow.int64())] - ) + pyschema = pyarrow.schema([('age', pyarrow.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", - file_name_suffix=".parquet" , + file_name_suffix=".parquet", num_shards=num_shards, - schema=pyschema - ) - output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + schema=pyschema) + output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp_xyz/ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet - # It captures: window_interval, shard_num, total_shards + # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) - - def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema( - [('age', pyarrow.int64())] - ) + pyschema = pyarrow.schema([('age', pyarrow.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", - file_name_suffix=".parquet" , + file_name_suffix=".parquet", shard_name_template=shard_name_template, num_shards=num_shards, - schema=pyschema - ) - output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + schema=pyschema) + output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) - def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, - num_shards = 2, - shard_name_template = '-V-SSSSS-of-NNNNN', - triggering_frequency = 5): + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema( - [('age', pyarrow.int64())] - ) + pyschema = pyarrow.schema([('age', pyarrow.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", file_name_suffix=".parquet", shard_name_template=shard_name_template, num_shards=num_shards, triggering_frequency=triggering_frequency, - schema=pyschema - ) - output2 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=True,level=logging.INFO) + schema=pyschema) + output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) #with 5s window size, the input should be processed by 5 windows with 2 shards per window - self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index c78601fc599e..3a1d6c94a03c 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -426,6 +426,7 @@ def output_type_hint(self): class _TextSourceWithFilename(_TextSource): + def read_records(self, file_name, range_tracker): records = super().read_records(file_name, range_tracker) for record in records: @@ -437,6 +438,7 @@ def output_type_hint(self): class _TextSink(filebasedsink.FileBasedSink): """A sink to a GCS or local text file or files.""" + def __init__( self, file_path_prefix, @@ -823,6 +825,7 @@ class ReadFromTextWithFilename(ReadFromText): class WriteToText(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for writing to text files.""" + def __init__( self, file_path_prefix: str, @@ -916,8 +919,10 @@ def expand(self, pcoll): if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE - self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) - self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) return pcoll | Write(self._sink) @@ -925,6 +930,7 @@ def expand(self, pcoll): import pandas def append_pandas_args(src, exclude): + def append(dest): state = None skip = False diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index fdd751e53ddd..d5e1e41ceba5 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -59,6 +59,7 @@ class DummyCoder(coders.Coder): + def encode(self, x): raise ValueError @@ -1482,6 +1483,7 @@ def test_read_escaped_escapechar_after_splitting_many(self): class TextSinkTest(unittest.TestCase): + def setUp(self): super().setUp() self.lines = [b'Line %d' % d for d in range(100)] @@ -1750,6 +1752,7 @@ def test_write_max_bytes_per_shard(self): class CsvTest(unittest.TestCase): + def test_csv_read_write(self): records = [beam.Row(a='str', b=ix) for ix in range(3)] with tempfile.TemporaryDirectory() as dest: @@ -1808,6 +1811,7 @@ def test_non_utf8_csv_read_write(self): class JsonTest(unittest.TestCase): + def test_json_read_write(self): records = [beam.Row(a='str', b=ix) for ix in range(3)] with tempfile.TemporaryDirectory() as dest: @@ -1853,45 +1857,121 @@ def check_types(element): _ = pcoll | beam.Map(check_types) + class GenerateEvent(beam.PTransform): - @staticmethod - def sample_data(): - return GenerateEvent() - - def expand(self, input): - elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - elem = elemlist - return (input - | TestStream() - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to_infinity()) + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + class WriteStreamingTest(unittest.TestCase): + def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -1900,66 +1980,76 @@ def tearDown(self): if os.path.exists(self.tempdir): shutil.rmtree(self.tempdir) - def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #TextIO output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( file_path_prefix=self.tempdir + "/ouput_WriteToText", file_name_suffix=".txt", num_shards=num_shards, - ) - output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + ) + output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp_xyz/ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt - # It captures: window_interval, shard_num, total_shards + # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) - - def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #TextIO output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( file_path_prefix=self.tempdir + "/ouput_WriteToText", file_name_suffix=".txt", shard_name_template=shard_name_template, num_shards=num_shards, - ) - output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + ) + output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) - def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, - num_shards = 2, - shard_name_template = '-V-SSSSS-of-NNNNN', - triggering_frequency = 5): + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data() - ) + output = (p | GenerateEvent.sample_data()) #TextIO output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( file_path_prefix=self.tempdir + "/ouput_WriteToText", @@ -1967,23 +2057,29 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, shard_name_template=shard_name_template, num_shards=num_shards, triggering_frequency=triggering_frequency, - ) - output2 | 'LogElements after WriteToText' >> LogElements(prefix='after WriteToText ', with_window=True,level=logging.INFO) + ) + output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) #with 5s window size, the input should be processed by 5 windows with 2 shards per window - self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index bd0bc127edd5..40d846ad6e43 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -85,6 +85,7 @@ class _TFRecordUtil(object): Note that masks and length are represented in LittleEndian order. """ + @classmethod def _masked_crc32c(cls, value, crc32c_fn=_default_crc32c_fn): """Compute a masked crc32c checksum for a value. @@ -176,6 +177,7 @@ class _TFRecordSource(FileBasedSource): For detailed TFRecords format description see: https://www.tensorflow.org/versions/r1.11/api_guides/python/python_io#TFRecords_Format_Details """ + def __init__(self, file_pattern, coder, compression_type, validate): """Initialize a TFRecordSource. See ReadFromTFRecord for details.""" super().__init__( @@ -212,6 +214,7 @@ def _create_tfrecordio_source( class ReadAllFromTFRecord(PTransform): """A ``PTransform`` for reading a ``PCollection`` of TFRecord files.""" + def __init__( self, coder=coders.BytesCoder(), @@ -249,6 +252,7 @@ def expand(self, pvalue): class ReadFromTFRecord(PTransform): """Transform for reading TFRecord sources.""" + def __init__( self, file_pattern, @@ -283,6 +287,7 @@ class _TFRecordSink(filebasedsink.FileBasedSink): For detailed TFRecord format description see: https://www.tensorflow.org/versions/r1.11/api_guides/python/python_io#TFRecords_Format_Details """ + def __init__( self, file_path_prefix, @@ -310,6 +315,7 @@ def write_encoded_record(self, file_handle, value): class WriteToTFRecord(PTransform): """Transform for writing to TFRecord sinks.""" + def __init__( self, file_path_prefix, @@ -359,7 +365,9 @@ def expand(self, pcoll): if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE - self._sink.shard_name_format = self._sink._template_to_format(self._sink.shard_name_template) - self._sink.shard_name_glob_format = self._sink._template_to_glob_format(self._sink.shard_name_template) + self._sink.shard_name_format = self._sink._template_to_format( + self._sink.shard_name_template) + self._sink.shard_name_glob_format = self._sink._template_to_glob_format( + self._sink.shard_name_template) return pcoll | Write(self._sink) diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index 0e48c8029dc6..c0ba186446d5 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -95,6 +95,7 @@ def _write_file_gzip(path, base64_records): class TestTFRecordUtil(unittest.TestCase): + def setUp(self): self.record = binascii.a2b_base64(FOO_RECORD_BASE64) @@ -166,6 +167,7 @@ def test_compatibility_read_write(self): class TestTFRecordSink(unittest.TestCase): + def _write_lines(self, sink, path, lines): f = sink.open(path) for l in lines: @@ -207,6 +209,7 @@ def test_write_record_multiple(self): @unittest.skipIf(tf is None, 'tensorflow not installed.') class TestWriteToTFRecord(TestTFRecordSink): + def test_write_record_gzip(self): with TempDir() as temp_dir: file_path_prefix = temp_dir.create_temp_file('result') @@ -243,6 +246,7 @@ def test_write_record_auto(self): class TestReadFromTFRecord(unittest.TestCase): + def test_process_single(self): with TempDir() as temp_dir: path = temp_dir.create_temp_file('result') @@ -335,6 +339,7 @@ def test_process_gzip_auto(self): class TestReadAllFromTFRecord(unittest.TestCase): + def _write_glob(self, temp_dir, suffix, include_empty=False): for _ in range(3): path = temp_dir.create_temp_file(suffix) @@ -475,6 +480,7 @@ def test_process_auto(self): class TestEnd2EndWriteAndRead(unittest.TestCase): + def create_inputs(self): input_array = [[random.random() - 0.5 for _ in range(15)] for _ in range(12)] @@ -564,45 +570,121 @@ def test_end2end_read_write_read(self): actual_data = p | ReadFromTFRecord(path + '-*', validate=True) assert_that(actual_data, equal_to(expected_data)) + class GenerateEvent(beam.PTransform): - @staticmethod - def sample_data(): - return GenerateEvent() - - def expand(self, input): - elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - elem = elemlist - return (input - | TestStream() - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 2, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 3, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 4, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 5, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 6, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 7, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 8, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 9, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 10, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 11, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 12, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 13, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 14, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 15, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 16, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 17, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 18, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 19, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .add_elements(elements=elem, event_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to(datetime(2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC).timestamp()) - .advance_watermark_to_infinity()) + @staticmethod + def sample_data(): + return GenerateEvent() + + def expand(self, input): + elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] + elem = elemlist + return ( + input + | TestStream().add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 2, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 3, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 4, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 5, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 6, + 0, tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 7, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 8, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 9, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 10, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 11, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 12, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 13, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 14, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 15, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 16, 0, + tzinfo=pytz.UTC).timestamp()). + add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 17, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 18, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 19, 0, + tzinfo=pytz.UTC).timestamp()). + advance_watermark_to( + datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).add_elements( + elements=elem, + event_timestamp=datetime( + 2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp()).advance_watermark_to( + datetime( + 2021, 3, 1, 0, 0, 25, 0, tzinfo=pytz.UTC). + timestamp()).advance_watermark_to_infinity()) + class WriteStreamingTest(unittest.TestCase): + def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -611,72 +693,85 @@ def tearDown(self): if os.path.exists(self.tempdir): shutil.rmtree(self.tempdir) - def test_write_streaming_2_shards_default_shard_name_template(self,num_shards = 2): + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): with TestPipeline() as p: - output = (p - | GenerateEvent.sample_data() - | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) - ) + output = ( + p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) #TFrecordIO output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", - file_name_suffix=".tfrecord" , - num_shards=num_shards, + file_name_suffix=".tfrecord", + num_shards=num_shards, ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp_xyz/ouput_WriteToTFRecord-[1614556800.0, 1614556805.0)-00000-of-00002.tfrecord - # It captures: window_interval, shard_num, total_shards + # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) - - def test_write_streaming_2_shards_custom_shard_name_template(self,num_shards = 2, shard_name_template = '-V-SSSSS-of-NNNNN'): + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template( + self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): with TestPipeline() as p: - output = (p - | GenerateEvent.sample_data() - | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) - ) + output = ( + p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) #TFrecordIO output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", - file_name_suffix=".tfrecord" , + file_name_suffix=".tfrecord", shard_name_template=shard_name_template, num_shards=num_shards, - ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + ) + output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - self.assertEqual(len(file_names), num_shards, "expected %d files, but got: %d" % (num_shards,len(file_names))) - - def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, - num_shards = 2, - shard_name_template = '-V-SSSSS-of-NNNNN', - triggering_frequency = 5): + self.assertEqual( + len(file_names), + num_shards, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + + def test_write_streaming_2_shards_custom_shard_name_template_5s_window( + self, + num_shards=2, + shard_name_template='-V-SSSSS-of-NNNNN', + triggering_frequency=5): with TestPipeline() as p: - output = (p - | GenerateEvent.sample_data() - | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8')) - ) + output = ( + p + | GenerateEvent.sample_data() + | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) #TFrecordIO output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( file_path_prefix=self.tempdir + "/ouput_WriteToTFRecord", @@ -684,23 +779,29 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window(self, shard_name_template=shard_name_template, num_shards=num_shards, triggering_frequency=triggering_frequency, - ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements(prefix='after WriteToTFRecord ', with_window=True,level=logging.INFO) + ) + output2 | 'LogElements after WriteToTFRecord' >> LogElements( + prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' - pattern = re.compile(pattern_string) + pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): match = pattern.match(file_name) - self.assertIsNotNone(match, f"File name {file_name} did not match expected pattern.") + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) #with 5s window size, the input should be processed by 5 windows with 2 shards per window - self.assertEqual(len(file_names), 10, "expected %d files, but got: %d" % (num_shards,len(file_names))) + self.assertEqual( + len(file_names), + 10, + "expected %d files, but got: %d" % (num_shards, len(file_names))) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From f01b84370b3eddc72513b65eda001267d4d868e2 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Tue, 29 Apr 2025 18:16:23 +0000 Subject: [PATCH 04/19] clean debug logs --- .../unbounded_sinks/test_write_bounded.py | 46 +++++++++---------- sdks/python/apache_beam/io/iobase.py | 30 +----------- 2 files changed, 24 insertions(+), 52 deletions(-) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py index 712364c61a99..79023f8e3dfa 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py @@ -71,11 +71,9 @@ def run(argv=None, save_main_session=True) -> PipelineResult: }, { 'age': 30 }]) - #| beam.CombineGlobally(AverageFn()) - #| 'Serialize' >> beam.Map(json.dumps) | beam.LogElements( prefix='before write ', with_window=False, level=logging.INFO)) - #OK in batch + #TextIO output2 = output | 'Write to text' >> WriteToText( file_path_prefix="__output_batch__/ouput_WriteToText", file_name_suffix=".txt", @@ -83,28 +81,28 @@ def run(argv=None, save_main_session=True) -> PipelineResult: output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=False, level=logging.INFO) - #OK in batch and stream - # output3 = (output | 'Serialize' >> beam.Map(json.dumps) - # | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") - # ) - # output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) + #FileIO + output3 = (output | 'Serialize' >> beam.Map(json.dumps) + | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") + ) + output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) - #KO - ValueError: GroupByKey cannot be applied to an unbounded PCollection with global windowing and a default trigger - # output4 = output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_parquet", - # schema= - # pyarrow.schema( - # [('age', pyarrow.int64())] - # ) - # ) - # output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) - # output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="output", - # schema= - # pyarrow.schema( - # [('cnt', pyarrow.int64()),('json', pyarrow.string())] - # ), - # record_batch_size = 10, - # num_shards=0 - # ) + #ParquetIO + output4 = output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_parquet", + schema= + pyarrow.schema( + [('age', pyarrow.int64())] + ) + ) + output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) + output | 'Write parquet' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_WriteToParquet", + schema= + pyarrow.schema( + [('age', pyarrow.int64())] + ), + record_batch_size = 10, + num_shards=0 + ) # Execute the pipeline and return the result. result = p.run() diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index abc58f2031f6..8cb56b85f045 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -1177,17 +1177,14 @@ def expand(self, pcoll): | 'Pair init gbk' >> core.GroupByKey() | 'InitializeWindowedWrite' >> core.Map( lambda _, sink: sink.initialize_write(), self.sink) - #| 'LogElements init_result_window_coll' >> LogElements(prefix="init_result_window_coll :",with_window=True,with_timestamp=True,level=logging.INFO) ) write_result_coll = ( keyed_pcoll | 'Group by random key' >> core.GroupByKey() - #| 'LogElements before WriteWindowedBundles' >> LogElements(prefix="before WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) | 'WriteWindowedBundles' >> core.ParDo( _WriteWindowedBundleDoFn(sink=self.sink, per_key=True), AsSingleton(init_result_window_coll)) - #| 'LogElements' >> LogElements(prefix="after WriteWindowedBundles :",with_window=True,with_timestamp=True,level=logging.INFO) | 'Pair' >> core.Map(lambda x: (None, x)) | core.GroupByKey() | 'Extract' >> core.Map(lambda x: x[1])) @@ -1198,7 +1195,6 @@ def expand(self, pcoll): AsSingleton(init_result_window_coll))) finalized_write_result_coll = ( pre_finalized_write_result_coll - #| 'LogElements pre_finalized_write_result_coll' >> LogElements(prefix="pre_finalized_write_result_coll :",with_window=True,with_timestamp=True,level=logging.INFO) | 'FinalizeWrite' >> core.FlatMap( _finalize_write, self.sink, @@ -1242,11 +1238,6 @@ def expand(self, pcoll): | 'WriteWindowedBundles' >> core.ParDo( _WriteWindowedBundleDoFn(self.sink), AsSingleton(init_result_window_coll)) - | 'LogElements' >> LogElements( - prefix="after WriteWindowedBundles :", - with_window=True, - with_timestamp=True, - level=logging.INFO) | 'Pair' >> core.Map(lambda x: (None, x)) | core.GroupByKey() | 'Extract' >> core.Map(lambda x: x[1])) @@ -1257,11 +1248,6 @@ def expand(self, pcoll): AsSingleton(init_result_window_coll))) finalized_write_result_coll = ( pre_finalized_write_result_coll - | 'LogElements 2' >> LogElements( - prefix="before finalize :", - with_window=True, - with_timestamp=True, - level=logging.INFO) | 'FinalizeWrite' >> core.FlatMap( _finalize_write, self.sink, @@ -1288,10 +1274,7 @@ def expand(self, pcoll): AsIter(write_result_coll), min_shards, AsSingleton(pre_finalize_coll)).with_output_types(str) - | 'LogElements after FinalizeWrite' >> LogElements( - prefix='after FinalizeWrite ', - with_window=False, - level=logging.INFO)) + ) class _WriteBundleDoFn(core.DoFn): @@ -1455,17 +1438,8 @@ def _finalize_write( outputs = sink.finalize_write( init_result, write_results + extra_shards, pre_finalize_results, w) outputs = list(outputs) - #_LOGGER.info("*** _finalize_write outputs %s",outputs) - if outputs: - if not isinstance(w, window.GlobalWindow): - #handle windowed finalize - yield ( - window.TimestampedValue(v, timestamp=self.window.start, windows=[w]) - for v in outputs) - else: - return ( - window.TimestampedValue(v, timestamp.MAX_TIMESTAMP) for v in outputs) + return (window.TimestampedValue(v, w.end) for v in outputs) class _RoundRobinKeyFn(core.DoFn): From d708e7c08ed8c71904a24e94b6f99db8d57d6e2a Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 12:02:16 +0000 Subject: [PATCH 05/19] moar yapf 0.29 and pylint --- .../unbounded_sinks/generate_event.py | 5 -- .../examples/unbounded_sinks/test_write.py | 20 +++----- .../unbounded_sinks/test_write_bounded.py | 45 ++++++++-------- sdks/python/apache_beam/io/avroio.py | 28 +++++----- sdks/python/apache_beam/io/avroio_test.py | 51 +++++++++++-------- sdks/python/apache_beam/io/filebasedsink.py | 35 ++++++++----- sdks/python/apache_beam/io/iobase.py | 28 ++-------- sdks/python/apache_beam/io/parquetio.py | 38 ++++++-------- sdks/python/apache_beam/io/parquetio_test.py | 39 +++++++++----- sdks/python/apache_beam/io/textio.py | 24 ++++----- sdks/python/apache_beam/io/textio_test.py | 35 ++++++++----- sdks/python/apache_beam/io/tfrecordio.py | 13 ++--- sdks/python/apache_beam/io/tfrecordio_test.py | 37 ++++++++------ 13 files changed, 196 insertions(+), 202 deletions(-) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py index 4065ce9a084e..7b21f227799e 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py @@ -18,20 +18,15 @@ from apache_beam.testing.test_stream import TestStream from datetime import datetime import pytz -import pyarrow class GenerateEvent(beam.PTransform): - @staticmethod def sample_data(): return GenerateEvent() def expand(self, input): elemlist = [{'age': 10}, {'age': 20}, {'age': 30}] - my_schema = pyarrow.schema([pyarrow.field('age', pyarrow.int64())], - metadata={"age": "the age int64"}) - #elem = [pyarrow.Table.from_pylist(elemlist, schema=my_schema)] elem = elemlist return ( input diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py index 39ee6ddbb7ed..43609cee3de2 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -17,13 +17,14 @@ # under the License. # -# python -m apache_beam.examples.unbounded_sinks.test_write --runner DirectRunner +# python -m apache_beam.examples.unbounded_sinks.test_write # This file contains multiple examples of writing unbounded PCollection to files -import apache_beam as beam, json, pyarrow +import apache_beam as beam import argparse +import json import logging -import re +import pyarrow from apache_beam.examples.unbounded_sinks.generate_event import GenerateEvent from apache_beam.transforms.window import FixedWindows from apache_beam.transforms.trigger import AccumulationMode @@ -37,7 +38,6 @@ class CountEvents(beam.PTransform): - def expand(self, events): return ( events @@ -62,12 +62,8 @@ def run(argv=None, save_main_session=True) -> PipelineResult: p = beam.Pipeline(options=pipeline_options) - output = ( - p | GenerateEvent.sample_data() - #| 'Count' >> CountEvents() - #| 'Serialize' >> beam.Map(json.dumps) - #| beam.LogElements(prefix='before write ', with_window=False, with_timestamp=True,level=logging.INFO) - ) + output = p | GenerateEvent.sample_data() + #TextIO output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( file_path_prefix="__output__/ouput_WriteToText", @@ -137,9 +133,9 @@ def run(argv=None, save_main_session=True) -> PipelineResult: #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension - 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'type': 'record', # type of avro serilazation, there are more (see above + # docs) but as per me this will do most of the time 'fields': [ # this defines actual keys & their types {'name': 'age', 'type': 'int'}, ], diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py index 79023f8e3dfa..c7b769d43ec0 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py @@ -17,14 +17,13 @@ # under the License. # -# python -m apache_beam.examples.unbounded_sinks.test_write --runner DirectRunner -# python -m apache_beam.examples.unbounded_sinks.test_write --region us-central1 --project ai-poney --temp_location=gs://poney-us --runner DataflowRunne +# python -m apache_beam.examples.unbounded_sinks.test_write -import apache_beam as beam, json, pyarrow +import apache_beam as beam import argparse +import json import logging -import re -from apache_beam.examples.unbounded_sinks.generate_event import GenerateEvent +import pyarrow from apache_beam.transforms.window import FixedWindows from apache_beam.transforms.trigger import AccumulationMode from apache_beam.transforms.trigger import AfterWatermark @@ -38,7 +37,6 @@ class CountEvents(beam.PTransform): - def expand(self, events): return ( events @@ -82,27 +80,24 @@ def run(argv=None, save_main_session=True) -> PipelineResult: prefix='after WriteToText ', with_window=False, level=logging.INFO) #FileIO - output3 = (output | 'Serialize' >> beam.Map(json.dumps) - | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles") - ) - output3 | 'LogElements after WriteToFiles' >> LogElements(prefix='after WriteToFiles ', with_window=False,level=logging.INFO) + output3 = ( + output | 'Serialize' >> beam.Map(json.dumps) + | 'Write to files' >> + WriteToFiles(path="__output_batch__/output_WriteToFiles")) + output3 | 'LogElements after WriteToFiles' >> LogElements( + prefix='after WriteToFiles ', with_window=False, level=logging.INFO) #ParquetIO - output4 = output | 'Write' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_parquet", - schema= - pyarrow.schema( - [('age', pyarrow.int64())] - ) - ) - output4 | 'LogElements after WriteToParquet' >> LogElements(prefix='after WriteToParquet ', with_window=False,level=logging.INFO) - output | 'Write parquet' >> beam.io.WriteToParquet(file_path_prefix="__output_batch__/output_WriteToParquet", - schema= - pyarrow.schema( - [('age', pyarrow.int64())] - ), - record_batch_size = 10, - num_shards=0 - ) + output4 = output | 'Write' >> beam.io.WriteToParquet( + file_path_prefix="__output_batch__/output_parquet", + schema=pyarrow.schema([('age', pyarrow.int64())])) + output4 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=False, level=logging.INFO) + output | 'Write parquet' >> beam.io.WriteToParquet( + file_path_prefix="__output_batch__/output_WriteToParquet", + schema=pyarrow.schema([('age', pyarrow.int64())]), + record_batch_size=10, + num_shards=0) # Execute the pipeline and return the result. result = p.run() diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 1b342d339355..457bb7dd4bf6 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -44,8 +44,6 @@ """ # pytype: skip-file import os -import pytz -import re from functools import partial from typing import Any from typing import Callable @@ -86,7 +84,6 @@ class ReadFromAvro(PTransform): that comply with the schema contained in the Avro file that contains those records. """ - def __init__( self, file_pattern=None, @@ -289,7 +286,6 @@ def expand(self, pbegin): class _AvroUtils(object): - @staticmethod def advance_file_past_next_sync_marker(f, sync_marker): buf_size = 10000 @@ -325,7 +321,6 @@ class _FastAvroSource(filebasedsource.FileBasedSource): TODO: remove ``_AvroSource`` in favor of using ``_FastAvroSource`` everywhere once it has been more widely tested """ - def read_records(self, file_name, range_tracker): next_block_start = -1 @@ -359,8 +354,7 @@ def split_points_unclaimed(stop_position): while range_tracker.try_claim(next_block_start): block = next(blocks) next_block_start = block.offset + block.size - for record in block: - yield record + yield from block _create_avro_source = _FastAvroSource @@ -371,7 +365,6 @@ class WriteToAvro(beam.transforms.PTransform): If the input has a schema, a corresponding avro schema will be automatically generated and used to write the output records.""" - def __init__( self, file_path_prefix, @@ -419,8 +412,14 @@ def __init__( """ self._schema = schema self._sink_provider = lambda avro_schema: _create_avro_sink( - file_path_prefix, avro_schema, codec, file_name_suffix, num_shards, - shard_name_template, mime_type, triggering_frequency) + file_path_prefix, + avro_schema, + codec, + file_name_suffix, + num_shards, + shard_name_template, + mime_type, + triggering_frequency) def expand(self, pcoll): if self._schema: @@ -437,9 +436,10 @@ def expand(self, pcoll): records = pcoll | beam.Map( beam_row_to_avro_dict(avro_schema, beam_schema)) self._sink = self._sink_provider(avro_schema) - if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: - # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format - self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + if (not pcoll.is_bounded and self._sink.shard_name_template == + filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) self._sink.shard_name_format = self._sink._template_to_format( self._sink.shard_name_template) self._sink.shard_name_glob_format = self._sink._template_to_glob_format( @@ -478,7 +478,6 @@ def _create_avro_sink( class _BaseAvroSink(filebasedsink.FileBasedSink): """A base for a sink for avro files. """ - def __init__( self, file_path_prefix, @@ -512,7 +511,6 @@ def display_data(self): class _FastAvroSink(_BaseAvroSink): """A sink for avro files using FastAvro. """ - def __init__( self, file_path_prefix, diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 2968c51c08fd..18cf1ea7a9ea 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -179,8 +179,8 @@ def test_schema_read_write(self): @pytest.mark.xlang_sql_expansion_service @unittest.skipIf( - TestPipeline().get_pipeline_options().view_as(StandardOptions).runner - is None, + TestPipeline().get_pipeline_options().view_as(StandardOptions).runner is + None, "Must be run with a runner that supports staging java artifacts.") def test_avro_schema_to_beam_schema_with_nullable_atomic_fields(self): records = [] @@ -381,7 +381,6 @@ def test_read_with_splitting_pattern(self): self._run_avro_test(pattern, 100, True, expected_result) def test_dynamic_work_rebalancing_exhaustive(self): - def compare_split_points(file_name): source = _FastAvroSource(file_name) splits = [ @@ -611,7 +610,6 @@ def test_writer_open_and_close(self): class TestFastAvro(AvroBase, unittest.TestCase): - def __init__(self, methodName='runTest'): super().__init__(methodName) self.SCHEMA = parse_schema(json.loads(self.SCHEMA_STRING)) @@ -635,7 +633,6 @@ def _write_data( class GenerateEvent(beam.PTransform): - @staticmethod def sample_data(): return GenerateEvent() @@ -747,7 +744,6 @@ def expand(self, input): class WriteStreamingTest(unittest.TestCase): - def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -762,9 +758,9 @@ def test_write_streaming_2_shards_default_shard_name_template( output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description - 'name': 'dummy', # your supposed to be file name with .avro extension - 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation, there are more (see + # above docs) 'fields': [ # this defines actual keys & their types {'name': 'age', 'type': 'int'}, ], @@ -778,9 +774,13 @@ def test_write_streaming_2_shards_default_shard_name_template( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp_xyz/ouput_WriteToAvro-[1614556800.0, 1614556805.0)-00000-of-00002.avro + # Example: + # ouput_WriteToAvro-[1614556800.0, 1614556805.0)-00000-of-00002.avro # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.avro$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): @@ -801,9 +801,8 @@ def test_write_streaming_2_shards_custom_shard_name_template( output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description - 'name': 'dummy', # your supposed to be file name with .avro extension - 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'name': 'dummy', # your supposed to be file name with .avro extension + 'type': 'record', # type of avro serilazation 'fields': [ # this defines actual keys & their types {'name': 'age', 'type': 'int'}, ], @@ -818,9 +817,14 @@ def test_write_streaming_2_shards_custom_shard_name_template( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro + # Example: + # ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.avro # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.avro$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.avro$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): @@ -844,9 +848,8 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( output = (p | GenerateEvent.sample_data()) #AvroIO avroschema = { - #'doc': 'A dummy avro file', # a short description 'name': 'dummy', # your supposed to be file name with .avro extension - 'type': 'record', # type of avro serilazation, there are more (see above docs) but as per me this will do most of the time + 'type': 'record', # type of avro serilazation 'fields': [ # this defines actual keys & their types {'name': 'age', 'type': 'int'}, ], @@ -862,9 +865,14 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.avro + # Example: + # ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.avro # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToAvro*'): @@ -874,7 +882,8 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - #with 5s window size, the input should be processed by 5 windows with 2 shards per window + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window self.assertEqual( len(file_names), 10, diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index aa7b6c35b644..f85577535476 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -38,7 +38,7 @@ DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' DEFAULT_WINDOW_SHARD_NAME_TEMPLATE = '-W-SSSSS-of-NNNNN' -DEFAULT_TRIGGERING_FREQUENCY = 60 +DEFAULT_TRIGGERING_FREQUENCY = 60 __all__ = ['FileBasedSink'] @@ -76,8 +76,7 @@ def __init__( max_bytes_per_shard=None, skip_if_empty=False, convert_fn=None, - triggering_frequency=None - ): + triggering_frequency=None): """ Raises: TypeError: if file path parameters are not a :class:`str` or @@ -217,11 +216,17 @@ def _get_final_name(self, shard_num, num_shards, w): if w is None or isinstance(w, window.GlobalWindow): window_utc = None else: - window_utc = '['+w.start.to_utc_datetime().isoformat()+', '+w.end.to_utc_datetime().isoformat()+')' + window_utc = ( + '[' + w.start.to_utc_datetime().isoformat() + ', ' + + w.end.to_utc_datetime().isoformat() + ')') return ''.join([ self.file_path_prefix.get(), - self.shard_name_format % - dict(shard_num=shard_num, num_shards=num_shards, uuid=(uuid.uuid4()), window=w, window_utc=window_utc), + self.shard_name_format % dict( + shard_num=shard_num, + num_shards=num_shards, + uuid=(uuid.uuid4()), + window=w, + window_utc=window_utc), self.file_name_suffix.get() ]) @@ -230,10 +235,16 @@ def _get_final_name_glob(self, num_shards, w): if w is None or isinstance(w, window.GlobalWindow): window_utc = None else: - window_utc = '['+w.start.to_utc_datetime().isoformat()+', '+w.end.to_utc_datetime().isoformat()+')' + window_utc = ( + '[' + w.start.to_utc_datetime().isoformat() + ', ' + + w.end.to_utc_datetime().isoformat() + ')') return ''.join([ self.file_path_prefix.get(), - self.shard_name_glob_format % dict(num_shards=num_shards, uuid=(uuid.uuid4()), window=w, window_utc=window_utc), + self.shard_name_glob_format % dict( + num_shards=num_shards, + uuid=(uuid.uuid4()), + window=w, + window_utc=window_utc), self.file_name_suffix.get() ]) @@ -252,7 +263,8 @@ def pre_finalize(self, init_result, writer_results, window=None): self.shard_name_glob_format) FileSystems.delete(dst_glob_files) - def _check_state_for_finalize_write(self, writer_results, num_shards, window=None): + def _check_state_for_finalize_write( + self, writer_results, num_shards, window=None): """Checks writer output files' states. Returns: @@ -376,7 +388,6 @@ def _rename_batch(batch): if w is None or isinstance(w, window.GlobalWindow): # bounded input # Use a thread pool for renaming operations. - exception_batches = util.run_using_threadpool( _rename_batch, list(zip(source_file_batch, destination_file_batch)), @@ -392,7 +403,6 @@ def _rename_batch(batch): yield from dst_files else: # unbounded input - #batch = list([source_file_batch, destination_file_batch]) batch = list([src_files, dst_files]) exception_batches = _rename_batch(batch) @@ -420,10 +430,7 @@ def _rename_batch(batch): except IOError: # This error is not serious, we simply log it. _LOGGER.info('Unable to delete file: %s', init_result) - - # return dst_files - @staticmethod def _template_replace_window(shard_name_template): match = re.search('W+', shard_name_template) diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index 8cb56b85f045..cc2a6d276652 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -60,7 +60,6 @@ from apache_beam.transforms.core import DoFn from apache_beam.transforms.display import DisplayDataItem from apache_beam.transforms.display import HasDisplayData -from apache_beam.transforms.util import LogElements from apache_beam.utils import timestamp from apache_beam.utils import urns from apache_beam.utils.windowed_value import WindowedValue @@ -146,7 +145,6 @@ class BoundedSource(SourceBase): implementations may invoke methods of ``BoundedSource`` objects through multi-threaded and/or reentrant execution modes. """ - def estimate_size(self) -> Optional[int]: """Estimates the size of source in bytes. @@ -850,7 +848,6 @@ class Writer(object): See ``iobase.Sink`` for more detailed documentation about the process of writing to a sink. """ - def write(self, value): """Writes a value to the sink using the current writer. """ @@ -1127,7 +1124,6 @@ def from_runner_api_parameter( class WriteImpl(ptransform.PTransform): """Implements the writing of custom sinks.""" - def __init__(self, sink: Sink) -> None: super().__init__() self.sink = sink @@ -1176,8 +1172,7 @@ def expand(self, pcoll): | 'Pair init' >> core.Map(lambda x: (None, x)) | 'Pair init gbk' >> core.GroupByKey() | 'InitializeWindowedWrite' >> core.Map( - lambda _, sink: sink.initialize_write(), self.sink) - ) + lambda _, sink: sink.initialize_write(), self.sink)) write_result_coll = ( keyed_pcoll @@ -1273,15 +1268,13 @@ def expand(self, pcoll): AsSingleton(init_result_coll), AsIter(write_result_coll), min_shards, - AsSingleton(pre_finalize_coll)).with_output_types(str) - ) + AsSingleton(pre_finalize_coll)).with_output_types(str)) class _WriteBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ - def __init__(self, sink): self.sink = sink @@ -1311,7 +1304,6 @@ class _PreFinalizeWindowedBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ - def __init__( self, sink, @@ -1340,7 +1332,6 @@ class _WriteWindowedBundleDoFn(core.DoFn): """A DoFn for writing elements to an iobase.Writer. Opens a writer at the first element and closes the writer at finish_bundle(). """ - def __init__(self, sink, per_key=False): self.sink = sink self.per_key = per_key @@ -1371,7 +1362,7 @@ def process( self.writer[w_key] = self.sink.open_writer( init_result, '%s_%s' % (w_key, uuid.uuid4())) self.init_result[w_key] = init_result - #_LOGGER.info("*** _WriteWindowedBundleDoFn writer %s", self.writer[w_key].temp_shard_path) + if self.per_key: for e in element[1]: # values self.writer[w_key].write(e) # value @@ -1399,7 +1390,6 @@ def finish_bundle(self): class _WriteKeyedBundleDoFn(core.DoFn): - def __init__(self, sink): self.sink = sink @@ -1441,8 +1431,8 @@ def _finalize_write( return (window.TimestampedValue(v, w.end) for v in outputs) -class _RoundRobinKeyFn(core.DoFn): +class _RoundRobinKeyFn(core.DoFn): def start_bundle(self): self.counter = None @@ -1466,7 +1456,6 @@ class RestrictionTracker(object): * https://s.apache.org/splittable-do-fn * https://s.apache.org/splittable-do-fn-python-sdk """ - def current_restriction(self): """Returns the current restriction. @@ -1601,7 +1590,6 @@ class WatermarkEstimator(object): Internal state must not be updated asynchronously. """ - def get_estimator_state(self): """Get current state of the WatermarkEstimator instance, which can be used to recreate the WatermarkEstimator when processing the restriction. See @@ -1627,7 +1615,6 @@ def observe_timestamp(self, timestamp: timestamp.Timestamp) -> None: class RestrictionProgress(object): """Used to record the progress of a restriction.""" - def __init__(self, **kwargs): # Only accept keyword arguments. self._fraction = kwargs.pop('fraction', None) @@ -1682,7 +1669,6 @@ def with_completed(self, completed: int) -> 'RestrictionProgress': class _SDFBoundedSourceRestriction(object): """ A restriction wraps SourceBundle and RangeTracker. """ - def __init__(self, source_bundle, range_tracker=None): self._source_bundle = source_bundle self._range_tracker = range_tracker @@ -1746,7 +1732,6 @@ class _SDFBoundedSourceRestrictionTracker(RestrictionTracker): Delegated RangeTracker guarantees synchronization safety. """ - def __init__(self, restriction): if not isinstance(restriction, _SDFBoundedSourceRestriction): raise ValueError( @@ -1783,7 +1768,6 @@ def is_bounded(self): class _SDFBoundedSourceWrapperRestrictionCoder(coders.Coder): - def decode(self, value): return _SDFBoundedSourceRestriction(SourceBundle(*pickler.loads(value))) @@ -1802,7 +1786,6 @@ class _SDFBoundedSourceRestrictionProvider(core.RestrictionProvider): This restriction provider initializes restriction based on input element that is expected to be of BoundedSource type. """ - def __init__(self, desired_chunk_size=None, restriction_coder=None): self._desired_chunk_size = desired_chunk_size self._restriction_coder = ( @@ -1852,15 +1835,12 @@ class SDFBoundedSourceReader(PTransform): NOTE: This transform can only be used with beam_fn_api enabled. """ - def __init__(self, data_to_display=None): self._data_to_display = data_to_display or {} super().__init__() def _create_sdf_bounded_source_dofn(self): - class SDFBoundedSourceDoFn(core.DoFn): - def __init__(self, dd): self._dd = dd diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 8c339def9630..fdad907ef8c1 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -77,7 +77,6 @@ class _ArrowTableToRowDictionaries(DoFn): """ A DoFn that consumes an Arrow table and yields a python dictionary for each row in the table.""" - def process(self, table, with_filename=False): if with_filename: file_name = table[0] @@ -96,7 +95,6 @@ def process(self, table, with_filename=False): class _RowDictionariesToArrowTable(DoFn): """ A DoFn that consumes python dictionarys and yields a pyarrow table.""" - def __init__( self, schema, @@ -169,7 +167,6 @@ def _flush_buffer(self): class _ArrowTableToBeamRows(DoFn): - def __init__(self, beam_type): self._beam_type = beam_type @@ -182,7 +179,6 @@ def infer_output_type(self, input_type): class _BeamRowsToArrowTable(DoFn): - @DoFn.yields_elements def process_batch(self, element: pa.Table) -> Iterator[pa.Table]: yield element @@ -192,7 +188,6 @@ class ReadFromParquetBatched(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is currently experimental. No backward-compatibility guarantees.""" - def __init__( self, file_pattern=None, min_bundle_size=0, validate=True, columns=None): """ Initializes :class:`~ReadFromParquetBatched` @@ -250,7 +245,6 @@ def display_data(self): class ReadFromParquet(PTransform): """A `PTransform` for reading Parquet files.""" - def __init__( self, file_pattern=None, @@ -389,7 +383,6 @@ def expand(self, pvalue): class ReadAllFromParquet(PTransform): - def __init__(self, with_filename=False, **kwargs): self._with_filename = with_filename self._read_batches = ReadAllFromParquetBatched( @@ -401,7 +394,6 @@ def expand(self, pvalue): class _ParquetUtils(object): - @staticmethod def find_first_row_group_index(pf, start_offset): for i in range(_ParquetUtils.get_number_of_row_groups(pf)): @@ -427,7 +419,6 @@ def get_number_of_row_groups(pf): class _ParquetSource(filebasedsource.FileBasedSource): """A source for reading Parquet files. """ - def __init__( self, file_pattern, min_bundle_size=0, validate=False, columns=None): super().__init__( @@ -486,7 +477,6 @@ def split_points_unclaimed(stop_position): class WriteToParquet(PTransform): """A ``PTransform`` for writing parquet files. """ - def __init__( self, file_path_prefix, @@ -599,9 +589,10 @@ def __init__( ) def expand(self, pcoll): - if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: - # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format - self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + if (not pcoll.is_bounded and self._sink.shard_name_template == + filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) self._sink.shard_name_format = self._sink._template_to_format( self._sink.shard_name_template) self._sink.shard_name_glob_format = self._sink._template_to_glob_format( @@ -639,7 +630,6 @@ class WriteToParquetBatched(PTransform): This ``PTransform`` is currently experimental. No backward-compatibility guarantees. """ - def __init__( self, file_path_prefix, @@ -709,18 +699,20 @@ def __init__( the shard number and shard count. When constructing a filename for a particular shard number, the upper-case letters 'S' and 'N' are replaced with the 0-padded shard number and shard count respectively. - W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` - V is used for windowed shard naming and is replaced with - ``[window.start.to_utc_datetime().isoformat(), window.end.to_utc_datetime().isoformat()`` + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().isoformat(), + window.end.to_utc_datetime().isoformat()`` This argument can be '' in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template and the PCollection is bounded. - The default pattern used is '-W-SSSSS-of-NNNNN' if None is passed as the + The default pattern used is '-W-SSSSS-of-NNNNN' if None is passed as the shard_name_template and the PCollection is unbounded. mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. - triggering_frequency: (int) Every triggering_frequency duration, a window + triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. Returns: @@ -742,9 +734,10 @@ def __init__( ) def expand(self, pcoll): - if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: - # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format - self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + if (not pcoll.is_bounded and self._sink.shard_name_template == + filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) self._sink.shard_name_format = self._sink._template_to_format( self._sink.shard_name_template) self._sink.shard_name_glob_format = self._sink._template_to_glob_format( @@ -783,7 +776,6 @@ def _create_parquet_sink( class _ParquetSink(filebasedsink.FileBasedSink): """A sink for parquet files from batches.""" - def __init__( self, file_path_prefix, diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index ff9311eea832..758c252302c6 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -29,7 +29,6 @@ import hamcrest as hc import pandas -import pyarrow import pytest from parameterized import param from parameterized import parameterized @@ -72,7 +71,6 @@ @unittest.skipIf(pa is None, "PyArrow is not installed.") @pytest.mark.uses_pyarrow class TestParquet(unittest.TestCase): - def setUp(self): # Reducing the size of thread pools. Without this test execution may fail in # environments with limited amount of resources. @@ -661,7 +659,6 @@ def test_read_all_from_parquet_with_filename(self): class GenerateEvent(beam.PTransform): - @staticmethod def sample_data(): return GenerateEvent() @@ -773,7 +770,6 @@ def expand(self, input): class WriteStreamingTest(unittest.TestCase): - def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -787,7 +783,7 @@ def test_write_streaming_2_shards_default_shard_name_template( with TestPipeline() as p: output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema([('age', pyarrow.int64())]) + pyschema = pa.schema([('age', pa.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", file_name_suffix=".parquet", @@ -797,9 +793,13 @@ def test_write_streaming_2_shards_default_shard_name_template( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp_xyz/ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet + # Example: + # ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): @@ -819,7 +819,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( with TestPipeline() as p: output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema([('age', pyarrow.int64())]) + pyschema = pa.schema([('age', pa.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", file_name_suffix=".parquet", @@ -830,9 +830,14 @@ def test_write_streaming_2_shards_custom_shard_name_template( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet + # Example: + # ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): @@ -855,7 +860,7 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( with TestPipeline() as p: output = (p | GenerateEvent.sample_data()) #ParquetIO - pyschema = pyarrow.schema([('age', pyarrow.int64())]) + pyschema = pa.schema([('age', pa.int64())]) output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", file_name_suffix=".parquet", @@ -867,9 +872,14 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.parquet + # Example: + # ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.parquet$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): @@ -879,7 +889,8 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - #with 5s window size, the input should be processed by 5 windows with 2 shards per window + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window self.assertEqual( len(file_names), 10, diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 3a1d6c94a03c..57b53f61d279 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -426,7 +426,6 @@ def output_type_hint(self): class _TextSourceWithFilename(_TextSource): - def read_records(self, file_name, range_tracker): records = super().read_records(file_name, range_tracker) for record in records: @@ -438,7 +437,6 @@ def output_type_hint(self): class _TextSink(filebasedsink.FileBasedSink): """A sink to a GCS or local text file or files.""" - def __init__( self, file_path_prefix, @@ -825,7 +823,6 @@ class ReadFromTextWithFilename(ReadFromText): class WriteToText(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for writing to text files.""" - def __init__( self, file_path_prefix: str, @@ -861,17 +858,20 @@ def __init__( unless you require a specific number of output files. shard_name_template (str): A template string containing placeholders for the shard number and shard count. Currently only ``''``, - ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and ``'-V-SSSSS-of-NNNNN'`` - are patterns accepted by the service. + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. When constructing a filename for a particular shard number, the upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded shard number and shard count respectively. This argument can be ``''`` in which case it behaves as if num_shards was set to 1 and only one file will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for - bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded PCollections. - W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` V is used for windowed shard naming and is replaced with - ``[window.start.to_utc_datetime().isoformat(), window.end.to_utc_datetime().isoformat()`` + ``[window.start.to_utc_datetime().isoformat(), + window.end.to_utc_datetime().isoformat()`` coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. compression_type (str): Used to handle compressed output files. Typical value is :class:`CompressionTypes.AUTO @@ -916,9 +916,10 @@ def __init__( triggering_frequency=triggering_frequency) def expand(self, pcoll): - if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: - # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format - self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + if (not pcoll.is_bounded and self._sink.shard_name_template == + filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) self._sink.shard_name_format = self._sink._template_to_format( self._sink.shard_name_template) self._sink.shard_name_glob_format = self._sink._template_to_glob_format( @@ -930,7 +931,6 @@ def expand(self, pcoll): import pandas def append_pandas_args(src, exclude): - def append(dest): state = None skip = False diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index d5e1e41ceba5..03026ab5dcee 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -59,7 +59,6 @@ class DummyCoder(coders.Coder): - def encode(self, x): raise ValueError @@ -1483,7 +1482,6 @@ def test_read_escaped_escapechar_after_splitting_many(self): class TextSinkTest(unittest.TestCase): - def setUp(self): super().setUp() self.lines = [b'Line %d' % d for d in range(100)] @@ -1752,7 +1750,6 @@ def test_write_max_bytes_per_shard(self): class CsvTest(unittest.TestCase): - def test_csv_read_write(self): records = [beam.Row(a='str', b=ix) for ix in range(3)] with tempfile.TemporaryDirectory() as dest: @@ -1811,7 +1808,6 @@ def test_non_utf8_csv_read_write(self): class JsonTest(unittest.TestCase): - def test_json_read_write(self): records = [beam.Row(a='str', b=ix) for ix in range(3)] with tempfile.TemporaryDirectory() as dest: @@ -1859,7 +1855,6 @@ def check_types(element): class GenerateEvent(beam.PTransform): - @staticmethod def sample_data(): return GenerateEvent() @@ -1971,7 +1966,6 @@ def expand(self, input): class WriteStreamingTest(unittest.TestCase): - def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -1994,9 +1988,13 @@ def test_write_streaming_2_shards_default_shard_name_template( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp_xyz/ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # Example: + # ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): @@ -2026,9 +2024,14 @@ def test_write_streaming_2_shards_custom_shard_name_template( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt + # Example: + # ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.txt # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): @@ -2062,9 +2065,14 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.txt + # Example: + # ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.txt # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.txt$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): @@ -2074,7 +2082,8 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - #with 5s window size, the input should be processed by 5 windows with 2 shards per window + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window self.assertEqual( len(file_names), 10, diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index 40d846ad6e43..fc472b1e9447 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -85,7 +85,6 @@ class _TFRecordUtil(object): Note that masks and length are represented in LittleEndian order. """ - @classmethod def _masked_crc32c(cls, value, crc32c_fn=_default_crc32c_fn): """Compute a masked crc32c checksum for a value. @@ -177,7 +176,6 @@ class _TFRecordSource(FileBasedSource): For detailed TFRecords format description see: https://www.tensorflow.org/versions/r1.11/api_guides/python/python_io#TFRecords_Format_Details """ - def __init__(self, file_pattern, coder, compression_type, validate): """Initialize a TFRecordSource. See ReadFromTFRecord for details.""" super().__init__( @@ -214,7 +212,6 @@ def _create_tfrecordio_source( class ReadAllFromTFRecord(PTransform): """A ``PTransform`` for reading a ``PCollection`` of TFRecord files.""" - def __init__( self, coder=coders.BytesCoder(), @@ -252,7 +249,6 @@ def expand(self, pvalue): class ReadFromTFRecord(PTransform): """Transform for reading TFRecord sources.""" - def __init__( self, file_pattern, @@ -287,7 +283,6 @@ class _TFRecordSink(filebasedsink.FileBasedSink): For detailed TFRecord format description see: https://www.tensorflow.org/versions/r1.11/api_guides/python/python_io#TFRecords_Format_Details """ - def __init__( self, file_path_prefix, @@ -315,7 +310,6 @@ def write_encoded_record(self, file_handle, value): class WriteToTFRecord(PTransform): """Transform for writing to TFRecord sinks.""" - def __init__( self, file_path_prefix, @@ -362,9 +356,10 @@ def __init__( triggering_frequency) def expand(self, pcoll): - if not pcoll.is_bounded and self._sink.shard_name_template == filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE: - # for unbounded PColl, change the default shard_name_template, shard_name_format and shard_name_glob_format - self._sink.shard_name_template = filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE + if (not pcoll.is_bounded and self._sink.shard_name_template == + filebasedsink.DEFAULT_SHARD_NAME_TEMPLATE): + self._sink.shard_name_template = ( + filebasedsink.DEFAULT_WINDOW_SHARD_NAME_TEMPLATE) self._sink.shard_name_format = self._sink._template_to_format( self._sink.shard_name_template) self._sink.shard_name_glob_format = self._sink._template_to_glob_format( diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index c0ba186446d5..5df20471b695 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -95,7 +95,6 @@ def _write_file_gzip(path, base64_records): class TestTFRecordUtil(unittest.TestCase): - def setUp(self): self.record = binascii.a2b_base64(FOO_RECORD_BASE64) @@ -167,7 +166,6 @@ def test_compatibility_read_write(self): class TestTFRecordSink(unittest.TestCase): - def _write_lines(self, sink, path, lines): f = sink.open(path) for l in lines: @@ -209,7 +207,6 @@ def test_write_record_multiple(self): @unittest.skipIf(tf is None, 'tensorflow not installed.') class TestWriteToTFRecord(TestTFRecordSink): - def test_write_record_gzip(self): with TempDir() as temp_dir: file_path_prefix = temp_dir.create_temp_file('result') @@ -246,7 +243,6 @@ def test_write_record_auto(self): class TestReadFromTFRecord(unittest.TestCase): - def test_process_single(self): with TempDir() as temp_dir: path = temp_dir.create_temp_file('result') @@ -339,7 +335,6 @@ def test_process_gzip_auto(self): class TestReadAllFromTFRecord(unittest.TestCase): - def _write_glob(self, temp_dir, suffix, include_empty=False): for _ in range(3): path = temp_dir.create_temp_file(suffix) @@ -480,7 +475,6 @@ def test_process_auto(self): class TestEnd2EndWriteAndRead(unittest.TestCase): - def create_inputs(self): input_array = [[random.random() - 0.5 for _ in range(15)] for _ in range(12)] @@ -572,7 +566,6 @@ def test_end2end_read_write_read(self): class GenerateEvent(beam.PTransform): - @staticmethod def sample_data(): return GenerateEvent() @@ -684,7 +677,6 @@ def expand(self, input): class WriteStreamingTest(unittest.TestCase): - def setUp(self): super().setUp() self.tempdir = tempfile.mkdtemp() @@ -710,9 +702,13 @@ def test_write_streaming_2_shards_default_shard_name_template( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp_xyz/ouput_WriteToTFRecord-[1614556800.0, 1614556805.0)-00000-of-00002.tfrecord + # Example: + # ouput_WriteToTFRecord-[1614556800.0, 1614556805.0)-00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P[\d\.]+), (?P[\d\.]+|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): @@ -745,9 +741,14 @@ def test_write_streaming_2_shards_custom_shard_name_template( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord + # Example: + # ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): @@ -784,9 +785,14 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern - # Example: /tmp/tmp7akb3opk/ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)-00000-of-00002.tfrecord + # Example: + # ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # 00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards - pattern_string = r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), (?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-(?P\d{5})-of-(?P\d{5})\.tfrecord$' + pattern_string = ( + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') pattern = re.compile(pattern_string) file_names = [] for file_name in glob.glob(self.tempdir + '/ouput_WriteToTFRecord*'): @@ -796,7 +802,8 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( if match: file_names.append(file_name) print("Found files matching expected pattern:", file_names) - #with 5s window size, the input should be processed by 5 windows with 2 shards per window + # for 5s window size, the input should be processed by 5 windows with + # 2 shards per window self.assertEqual( len(file_names), 10, From 2d92cab6ccd4d719df694012275640b26fb5f615 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 14:42:29 +0000 Subject: [PATCH 06/19] revert a change interating over None in iobase._finalize_write fix write_ptransform_test.py fix iobase.Sink.finalize_write params fix lint warnings reorder imports --- .../unbounded_sinks/generate_event.py | 6 ++-- .../examples/unbounded_sinks/test_write.py | 31 ++++++++++--------- .../unbounded_sinks/test_write_bounded.py | 26 +++++++++------- sdks/python/apache_beam/io/avroio_test.py | 6 ++-- sdks/python/apache_beam/io/filebasedsink.py | 7 ----- sdks/python/apache_beam/io/iobase.py | 9 +++--- sdks/python/apache_beam/io/parquetio_test.py | 12 +++---- sdks/python/apache_beam/io/textio_test.py | 17 +++++----- sdks/python/apache_beam/io/tfrecordio_test.py | 10 +++--- .../transforms/write_ptransform_test.py | 6 +++- 10 files changed, 68 insertions(+), 62 deletions(-) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py index 7b21f227799e..5b23e43fc368 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/generate_event.py @@ -14,11 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. -import apache_beam as beam -from apache_beam.testing.test_stream import TestStream from datetime import datetime + import pytz +import apache_beam as beam +from apache_beam.testing.test_stream import TestStream + class GenerateEvent(beam.PTransform): @staticmethod diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py index 43609cee3de2..7390d836c33c 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -20,21 +20,24 @@ # python -m apache_beam.examples.unbounded_sinks.test_write # This file contains multiple examples of writing unbounded PCollection to files -import apache_beam as beam + import argparse import json import logging + import pyarrow + +import apache_beam as beam from apache_beam.examples.unbounded_sinks.generate_event import GenerateEvent -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.trigger import AccumulationMode -from apache_beam.transforms.trigger import AfterWatermark -from apache_beam.utils.timestamp import Duration -from apache_beam.transforms.util import LogElements from apache_beam.io.fileio import WriteToFiles from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions from apache_beam.runners.runner import PipelineResult +from apache_beam.transforms.trigger import AccumulationMode +from apache_beam.transforms.trigger import AfterWatermark +from apache_beam.transforms.util import LogElements +from apache_beam.transforms.window import FixedWindows +from apache_beam.utils.timestamp import Duration class CountEvents(beam.PTransform): @@ -53,7 +56,7 @@ def expand(self, events): def run(argv=None, save_main_session=True) -> PipelineResult: """Main entry point; defines and runs the wordcount pipeline.""" parser = argparse.ArgumentParser() - known_args, pipeline_args = parser.parse_known_args(argv) + _, pipeline_args = parser.parse_known_args(argv) # We use the save_main_session option because one or more DoFn's in this # workflow rely on global context (e.g., a module imported at module level). @@ -71,11 +74,11 @@ def run(argv=None, save_main_session=True) -> PipelineResult: #shard_name_template='-V-SSSSS-of-NNNNN', num_shards=2, #triggering_frequency=5, ) - output2 | 'LogElements after WriteToText' >> LogElements( + _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) #FileIO - output5 = ( + _ = ( output | 'FileIO window' >> beam.WindowInto( FixedWindows(5), @@ -96,7 +99,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: num_shards=2, triggering_frequency=5, schema=pyschema) - output4a | 'LogElements after WriteToParquet' >> LogElements( + _ = output4a | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet 4a ', with_window=True, level=logging.INFO) output4aw = ( @@ -112,7 +115,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: file_name_suffix=".parquet", num_shards=2, schema=pyschema)) - output4aw | 'LogElements after WriteToParquet windowed' >> LogElements( + _ = output4aw | 'LogElements after WriteToParquet windowed' >> LogElements( prefix='after WriteToParquet 4aw ', with_window=True, level=logging.INFO) output4b = ( @@ -126,7 +129,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: num_shards=2, triggering_frequency=5, schema=pyschema)) - output4b | 'LogElements after WriteToParquetBatched' >> LogElements( + _ = output4b | 'LogElements after WriteToParquetBatched' >> LogElements( prefix='after WriteToParquetBatched 4b ', with_window=True, level=logging.INFO) @@ -147,7 +150,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: num_shards=2, #triggering_frequency=5, schema=avroschema) - output5 | 'LogElements after WriteToAvro' >> LogElements( + _ = output5 | 'LogElements after WriteToAvro' >> LogElements( prefix='after WriteToAvro 5 ', with_window=True, level=logging.INFO) #TFrecordIO @@ -160,7 +163,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: file_name_suffix=".tfrecord", num_shards=2, triggering_frequency=5)) - output6 | 'LogElements after WriteToTFRecord' >> LogElements( + _ = output6 | 'LogElements after WriteToTFRecord' >> LogElements( prefix='after WriteToTFRecord 6 ', with_window=True, level=logging.INFO) # Execute the pipeline and return the result. diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py index c7b769d43ec0..7e24dc433de5 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write_bounded.py @@ -19,21 +19,23 @@ # python -m apache_beam.examples.unbounded_sinks.test_write -import apache_beam as beam import argparse import json import logging + import pyarrow -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.trigger import AccumulationMode -from apache_beam.transforms.trigger import AfterWatermark -from apache_beam.utils.timestamp import Duration -from apache_beam.transforms.util import LogElements -from apache_beam.io.textio import WriteToText + +import apache_beam as beam from apache_beam.io.fileio import WriteToFiles +from apache_beam.io.textio import WriteToText from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions from apache_beam.runners.runner import PipelineResult +from apache_beam.transforms.trigger import AccumulationMode +from apache_beam.transforms.trigger import AfterWatermark +from apache_beam.transforms.util import LogElements +from apache_beam.transforms.window import FixedWindows +from apache_beam.utils.timestamp import Duration class CountEvents(beam.PTransform): @@ -52,7 +54,7 @@ def expand(self, events): def run(argv=None, save_main_session=True) -> PipelineResult: """Main entry point; defines and runs the wordcount pipeline.""" parser = argparse.ArgumentParser() - known_args, pipeline_args = parser.parse_known_args(argv) + _, pipeline_args = parser.parse_known_args(argv) # We use the save_main_session option because one or more DoFn's in this # workflow rely on global context (e.g., a module imported at module level). @@ -76,7 +78,7 @@ def run(argv=None, save_main_session=True) -> PipelineResult: file_path_prefix="__output_batch__/ouput_WriteToText", file_name_suffix=".txt", shard_name_template='-U-SSSSS-of-NNNNN') - output2 | 'LogElements after WriteToText' >> LogElements( + _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=False, level=logging.INFO) #FileIO @@ -84,16 +86,16 @@ def run(argv=None, save_main_session=True) -> PipelineResult: output | 'Serialize' >> beam.Map(json.dumps) | 'Write to files' >> WriteToFiles(path="__output_batch__/output_WriteToFiles")) - output3 | 'LogElements after WriteToFiles' >> LogElements( + _ = output3 | 'LogElements after WriteToFiles' >> LogElements( prefix='after WriteToFiles ', with_window=False, level=logging.INFO) #ParquetIO output4 = output | 'Write' >> beam.io.WriteToParquet( file_path_prefix="__output_batch__/output_parquet", schema=pyarrow.schema([('age', pyarrow.int64())])) - output4 | 'LogElements after WriteToParquet' >> LogElements( + _ = output4 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=False, level=logging.INFO) - output | 'Write parquet' >> beam.io.WriteToParquet( + _ = output | 'Write parquet' >> beam.io.WriteToParquet( file_path_prefix="__output_batch__/output_WriteToParquet", schema=pyarrow.schema([('age', pyarrow.int64())]), record_batch_size=10, diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 18cf1ea7a9ea..655adc995e60 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -770,7 +770,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_name_suffix=".avro", num_shards=num_shards, schema=avroschema) - output2 | 'LogElements after WriteToAvro' >> LogElements( + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -813,7 +813,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( shard_name_template=shard_name_template, num_shards=num_shards, schema=avroschema) - output2 | 'LogElements after WriteToAvro' >> LogElements( + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -861,7 +861,7 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( num_shards=num_shards, triggering_frequency=triggering_frequency, schema=avroschema) - output2 | 'LogElements after WriteToAvro' >> LogElements( + _ = output2 | 'LogElements after WriteToAvro' >> LogElements( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index f85577535476..84f6f97f85a9 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -353,13 +353,6 @@ def finalize_write( ] if num_shards_to_finalize: - # _LOGGER.info( - # 'Starting finalize_write threads with num_shards: %d (skipped: %d), ' - # 'batches: %d, num_threads: %d', - # num_shards_to_finalize, - # num_skipped, - # len(source_file_batch), - # num_threads) start_time = time.time() def _rename_batch(batch): diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index cc2a6d276652..e08c75cea0ae 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -35,7 +35,6 @@ import math import random import uuid -import apache_beam as beam from collections import namedtuple from typing import Any from typing import Iterator @@ -43,6 +42,7 @@ from typing import Tuple from typing import Union +import apache_beam as beam from apache_beam import coders from apache_beam import pvalue from apache_beam.coders.coders import _MemoizingPickleCoder @@ -799,7 +799,7 @@ def pre_finalize(self, init_result, writer_results, window=None): """ raise NotImplementedError - def finalize_write(self, init_result, writer_results, pre_finalize_result): + def finalize_write(self, init_result, writer_results, pre_finalize_result, w): """Finalizes the sink after all data is written to it. Given the result of initialization and an iterable of results from bundle @@ -832,6 +832,7 @@ def finalize_write(self, init_result, writer_results, pre_finalize_result): will only contain the result of a single successful write for a given bundle. pre_finalize_result: the result of ``pre_finalize()`` invocation. + w: DoFn window """ raise NotImplementedError @@ -1427,9 +1428,9 @@ def _finalize_write( extra_shards.append(writer.close()) outputs = sink.finalize_write( init_result, write_results + extra_shards, pre_finalize_results, w) - outputs = list(outputs) - return (window.TimestampedValue(v, w.end) for v in outputs) + if outputs: + return (window.TimestampedValue(v, w.end) for v in outputs) class _RoundRobinKeyFn(core.DoFn): diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index 758c252302c6..02bd25f26732 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -16,20 +16,21 @@ # # pytype: skip-file -import json import glob +import json import logging import os -import pytz import re import shutil import tempfile import unittest +from datetime import datetime from tempfile import TemporaryDirectory import hamcrest as hc import pandas import pytest +import pytz from parameterized import param from parameterized import parameterized @@ -54,7 +55,6 @@ from apache_beam.transforms.display import DisplayData from apache_beam.transforms.display_test import DisplayDataItemMatcher from apache_beam.transforms.util import LogElements -from datetime import datetime try: import pyarrow as pa @@ -789,7 +789,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_name_suffix=".parquet", num_shards=num_shards, schema=pyschema) - output2 | 'LogElements after WriteToParquet' >> LogElements( + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -826,7 +826,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( shard_name_template=shard_name_template, num_shards=num_shards, schema=pyschema) - output2 | 'LogElements after WriteToParquet' >> LogElements( + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -868,7 +868,7 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( num_shards=num_shards, triggering_frequency=triggering_frequency, schema=pyschema) - output2 | 'LogElements after WriteToParquet' >> LogElements( + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index 03026ab5dcee..756f48b657bd 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -23,13 +23,15 @@ import gzip import logging import os -import pytz -import re import platform +import re import shutil import tempfile import unittest import zlib +from datetime import datetime + +import pytz import apache_beam as beam from apache_beam import coders @@ -47,15 +49,14 @@ from apache_beam.io.textio import WriteToText from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.testing.test_pipeline import TestPipeline -from apache_beam.testing.test_utils import TempDir from apache_beam.testing.test_stream import TestStream +from apache_beam.testing.test_utils import TempDir from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.core import Create -from apache_beam.transforms.util import LogElements from apache_beam.transforms.userstate import CombiningValueStateSpec +from apache_beam.transforms.util import LogElements from apache_beam.utils.timestamp import Timestamp -from datetime import datetime class DummyCoder(coders.Coder): @@ -1984,7 +1985,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_name_suffix=".txt", num_shards=num_shards, ) - output2 | 'LogElements after WriteToText' >> LogElements( + _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -2020,7 +2021,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( shard_name_template=shard_name_template, num_shards=num_shards, ) - output2 | 'LogElements after WriteToText' >> LogElements( + _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -2061,7 +2062,7 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( num_shards=num_shards, triggering_frequency=triggering_frequency, ) - output2 | 'LogElements after WriteToText' >> LogElements( + _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index 5df20471b695..b694ccc6b782 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -25,15 +25,16 @@ import logging import os import pickle -import pytz import random import re import shutil import tempfile import unittest import zlib +from datetime import datetime import crcmod +import pytz import apache_beam as beam from apache_beam import Create @@ -50,7 +51,6 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.transforms.util import LogElements -from datetime import datetime try: import tensorflow.compat.v1 as tf # pylint: disable=import-error @@ -698,7 +698,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_name_suffix=".tfrecord", num_shards=num_shards, ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements( + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -737,7 +737,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( shard_name_template=shard_name_template, num_shards=num_shards, ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements( + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern @@ -781,7 +781,7 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( num_shards=num_shards, triggering_frequency=triggering_frequency, ) - output2 | 'LogElements after WriteToTFRecord' >> LogElements( + _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO) # Regex to match the expected windowed file pattern diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index ce402d8d3062..0a63060c4073 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -45,7 +45,11 @@ def pre_finalize(self, init_result, writer_results): pass def finalize_write( - self, init_result, writer_results, unused_pre_finalize_result): + self, + init_result, + writer_results, + unused_pre_finalize_result, + unused_window): self.init_result_at_finalize = init_result self.write_results_at_finalize = writer_results From ac46d4381e15961e432f36b5bdad04d99cc632df Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 15:20:55 +0000 Subject: [PATCH 07/19] fix test sink in snippets --- sdks/python/apache_beam/examples/snippets/snippets.py | 7 ++++++- sdks/python/apache_beam/io/iobase.py | 2 +- .../python/apache_beam/transforms/write_ptransform_test.py | 2 +- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index f6bf5e5d44ec..34e80bb3d72f 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -698,7 +698,12 @@ def open_writer(self, access_token, uid): def pre_finalize(self, init_result, writer_results): pass - def finalize_write(self, access_token, table_names, pre_finalize_result): + def finalize_write( + self, + access_token, + table_names, + pre_finalize_result, + unused_window): for i, table_name in enumerate(table_names): self._simplekv.rename_table( access_token, table_name, self._final_table_name + str(i)) diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index e08c75cea0ae..adc880170157 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -799,7 +799,7 @@ def pre_finalize(self, init_result, writer_results, window=None): """ raise NotImplementedError - def finalize_write(self, init_result, writer_results, pre_finalize_result, w): + def finalize_write(self, init_result, writer_results, pre_finalize_result, w=None): """Finalizes the sink after all data is written to it. Given the result of initialization and an iterable of results from bundle diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index 0a63060c4073..459d404a5a95 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -49,7 +49,7 @@ def finalize_write( init_result, writer_results, unused_pre_finalize_result, - unused_window): + unused_w): self.init_result_at_finalize = init_result self.write_results_at_finalize = writer_results From f13a3e30de89de7cd7c1f09822d91e07ff8b216e Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 15:22:16 +0000 Subject: [PATCH 08/19] add more detail on IOBase.sink change --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index 82edaac24060..6fc4deb5e1f5 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -71,6 +71,7 @@ * Upgraded GoogleAdsAPI to v19 for GoogleAdsIO (Java) ([#34497](https://github.com/apache/beam/pull/34497)). Changed PTransform method from version-specified (`v17()`) to `current()` for better backward compatibility in the future. * Added support for writing to Pubsub with ordering keys (Java) ([#21162](https://github.com/apache/beam/issues/21162)) * Support for streaming writes for AvroIO, ParquetIO, TextIO, TFRecordIO +* IOBase.Sink finalize_write has a new optional parameter w for the window ## New Features / Improvements From c4e57f2e29e14c6ece657293ec8f19d6cfeedb54 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 15:37:57 +0000 Subject: [PATCH 09/19] text formatting fix --- sdks/python/apache_beam/examples/snippets/snippets.py | 6 +----- .../apache_beam/examples/unbounded_sinks/test_write.py | 1 - sdks/python/apache_beam/io/iobase.py | 3 ++- sdks/python/apache_beam/transforms/write_ptransform_test.py | 6 +----- 4 files changed, 4 insertions(+), 12 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 34e80bb3d72f..08a092355c1c 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -699,11 +699,7 @@ def pre_finalize(self, init_result, writer_results): pass def finalize_write( - self, - access_token, - table_names, - pre_finalize_result, - unused_window): + self, access_token, table_names, pre_finalize_result, unused_window): for i, table_name in enumerate(table_names): self._simplekv.rename_table( access_token, table_name, self._final_table_name + str(i)) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py index 7390d836c33c..5b52f063b513 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -20,7 +20,6 @@ # python -m apache_beam.examples.unbounded_sinks.test_write # This file contains multiple examples of writing unbounded PCollection to files - import argparse import json import logging diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index adc880170157..1b3fdaed1f37 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -799,7 +799,8 @@ def pre_finalize(self, init_result, writer_results, window=None): """ raise NotImplementedError - def finalize_write(self, init_result, writer_results, pre_finalize_result, w=None): + def finalize_write( + self, init_result, writer_results, pre_finalize_result, w=None): """Finalizes the sink after all data is written to it. Given the result of initialization and an iterable of results from bundle diff --git a/sdks/python/apache_beam/transforms/write_ptransform_test.py b/sdks/python/apache_beam/transforms/write_ptransform_test.py index 459d404a5a95..8525adb4b74a 100644 --- a/sdks/python/apache_beam/transforms/write_ptransform_test.py +++ b/sdks/python/apache_beam/transforms/write_ptransform_test.py @@ -45,11 +45,7 @@ def pre_finalize(self, init_result, writer_results): pass def finalize_write( - self, - init_result, - writer_results, - unused_pre_finalize_result, - unused_w): + self, init_result, writer_results, unused_pre_finalize_result, unused_w): self.init_result_at_finalize = init_result self.write_results_at_finalize = writer_results From afb69c7dd089f89cfa4873b212c03a7eacc9ae0c Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 16:34:06 +0000 Subject: [PATCH 10/19] make triggering_frequent optional in _create_parquet_sink --- sdks/python/apache_beam/io/parquetio.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index fdad907ef8c1..6f36285813a3 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -758,7 +758,7 @@ def _create_parquet_sink( num_shards, shard_name_template, mime_type, - triggering_frequency): + triggering_frequency=None): return \ _ParquetSink( file_path_prefix, From 5bd5e15eb85124ddd12e895c33ab4720d25a9b93 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 17:13:23 +0000 Subject: [PATCH 11/19] make triggering_frequency opt in _TFRecordSink , _create_avro_sink, _create_parquet_sink --- sdks/python/apache_beam/io/avroio.py | 2 +- sdks/python/apache_beam/io/parquetio.py | 2 +- sdks/python/apache_beam/io/tfrecordio.py | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 457bb7dd4bf6..c47cf9216977 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -459,7 +459,7 @@ def _create_avro_sink( num_shards, shard_name_template, mime_type, - triggering_frequency): + triggering_frequency=60): if "class 'avro.schema" in str(type(schema)): raise ValueError( 'You are using Avro IO with fastavro (default with Beam on ' diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 6f36285813a3..a53720bc6499 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -758,7 +758,7 @@ def _create_parquet_sink( num_shards, shard_name_template, mime_type, - triggering_frequency=None): + triggering_frequency=60): return \ _ParquetSink( file_path_prefix, diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index fc472b1e9447..a75ebcc4dc30 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -353,7 +353,7 @@ def __init__( num_shards, shard_name_template, compression_type, - triggering_frequency) + triggering_frequency=60) def expand(self, pcoll): if (not pcoll.is_bounded and self._sink.shard_name_template == From 1bd138c3016dc8dcea8548dd83160a86ae140692 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 17:49:26 +0000 Subject: [PATCH 12/19] fix _TFRecordSink making triggering_frequency opt --- sdks/python/apache_beam/io/tfrecordio.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index a75ebcc4dc30..31cd752f4bb9 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -291,7 +291,7 @@ def __init__( num_shards, shard_name_template, compression_type, - triggering_frequency): + triggering_frequency=60): """Initialize a TFRecordSink. See WriteToTFRecord for details.""" super().__init__( @@ -353,7 +353,7 @@ def __init__( num_shards, shard_name_template, compression_type, - triggering_frequency=60) + triggering_frequency) def expand(self, pcoll): if (not pcoll.is_bounded and self._sink.shard_name_template == From 491a285fee4d193006e9e7310a356b024bde2111 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 30 Apr 2025 19:29:59 +0000 Subject: [PATCH 13/19] fix filenaming format for windows (no ":") update doc about filenaming in streaming --- sdks/python/apache_beam/io/avroio.py | 21 +++++++++++++------ sdks/python/apache_beam/io/avroio_test.py | 12 +++++------ sdks/python/apache_beam/io/filebasedsink.py | 8 +++---- sdks/python/apache_beam/io/parquetio.py | 21 +++++++++++++------ sdks/python/apache_beam/io/parquetio_test.py | 12 +++++------ sdks/python/apache_beam/io/textio.py | 4 ++-- sdks/python/apache_beam/io/textio_test.py | 12 +++++------ sdks/python/apache_beam/io/tfrecordio.py | 21 +++++++++++++------ sdks/python/apache_beam/io/tfrecordio_test.py | 12 +++++------ 9 files changed, 75 insertions(+), 48 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index c47cf9216977..78f70d402db4 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -394,12 +394,21 @@ def __init__( the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. use_fastavro (bool): This flag is left for API backwards compatibility diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 655adc995e60..20d4f0725bcc 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -818,12 +818,12 @@ def test_write_streaming_2_shards_custom_shard_name_template( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToAvro-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.avro # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.avro$') pattern = re.compile(pattern_string) file_names = [] @@ -866,12 +866,12 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToAvro-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToAvro-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.avro # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index 84f6f97f85a9..447e1c19dd58 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -217,8 +217,8 @@ def _get_final_name(self, shard_num, num_shards, w): window_utc = None else: window_utc = ( - '[' + w.start.to_utc_datetime().isoformat() + ', ' + - w.end.to_utc_datetime().isoformat() + ')') + '[' + w.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S") + ', ' + + w.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S") + ')') return ''.join([ self.file_path_prefix.get(), self.shard_name_format % dict( @@ -236,8 +236,8 @@ def _get_final_name_glob(self, num_shards, w): window_utc = None else: window_utc = ( - '[' + w.start.to_utc_datetime().isoformat() + ', ' + - w.end.to_utc_datetime().isoformat() + ')') + '[' + w.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S") + ', ' + + w.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S") + ')') return ''.join([ self.file_path_prefix.get(), self.shard_name_glob_format % dict( diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index a53720bc6499..cee467caa777 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -555,12 +555,21 @@ def __init__( the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` mime_type: The MIME type to use for the produced files, if the filesystem supports specifying MIME types. triggering_frequency: (int) Every triggering_frequency duration, a window diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index 02bd25f26732..ef8e7b8ab940 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -831,12 +831,12 @@ def test_write_streaming_2_shards_custom_shard_name_template( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.parquet$') pattern = re.compile(pattern_string) file_names = [] @@ -873,12 +873,12 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToParquet-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToParquet-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.parquet # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.parquet$') pattern = re.compile(pattern_string) file_names = [] diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 57b53f61d279..c26a01da5d86 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -870,8 +870,8 @@ def __init__( W is used for windowed shard naming and is replaced with ``[window.start, window.end)`` V is used for windowed shard naming and is replaced with - ``[window.start.to_utc_datetime().isoformat(), - window.end.to_utc_datetime().isoformat()`` + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. compression_type (str): Used to handle compressed output files. Typical value is :class:`CompressionTypes.AUTO diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index 756f48b657bd..e88893d5e669 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -2026,12 +2026,12 @@ def test_write_streaming_2_shards_custom_shard_name_template( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToText-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.txt # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] @@ -2067,12 +2067,12 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToText-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToText-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.txt # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.txt$') pattern = re.compile(pattern_string) file_names = [] diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index 31cd752f4bb9..57ca00e38c05 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -330,12 +330,21 @@ def __init__( num_shards: The number of files (shards) used for output. If not set, the default value will be used. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` compression_type: Used to handle compressed output files. Typical value is CompressionTypes.AUTO, in which case the file_path's extension will be used to detect the compression. diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index b694ccc6b782..5875e4511809 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -742,12 +742,12 @@ def test_write_streaming_2_shards_custom_shard_name_template( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToTFRecord-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') pattern = re.compile(pattern_string) file_names = [] @@ -786,12 +786,12 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( # Regex to match the expected windowed file pattern # Example: - # ouput_WriteToTFRecord-[2021-03-01T00:00:00, 2021-03-01T00:01:00)- + # ouput_WriteToTFRecord-[2021-03-01T00-00-00, 2021-03-01T00-01-00)- # 00000-of-00002.tfrecord # It captures: window_interval, shard_num, total_shards pattern_string = ( - r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}), ' - r'(?P\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}|Infinity)\)-' + r'.*-\[(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}), ' + r'(?P\d{4}-\d{2}-\d{2}T\d{2}-\d{2}-\d{2}|Infinity)\)-' r'(?P\d{5})-of-(?P\d{5})\.tfrecord$') pattern = re.compile(pattern_string) file_names = [] From 9ca2f668930bd5eeb8581e15e77aba2d8de20516 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Mon, 5 May 2025 08:17:53 +0000 Subject: [PATCH 14/19] add iobase_it_test from PR 34814 --- sdks/python/apache_beam/io/iobase_it_test.py | 72 ++++++++++++++++++++ 1 file changed, 72 insertions(+) create mode 100644 sdks/python/apache_beam/io/iobase_it_test.py diff --git a/sdks/python/apache_beam/io/iobase_it_test.py b/sdks/python/apache_beam/io/iobase_it_test.py new file mode 100644 index 000000000000..f63145ef046a --- /dev/null +++ b/sdks/python/apache_beam/io/iobase_it_test.py @@ -0,0 +1,72 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# pytype: skip-file + +import logging +import unittest +import uuid + +import apache_beam as beam +from apache_beam.io.textio import WriteToText +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.transforms.window import FixedWindows + +# End-to-End tests for iobase +# Usage: +# cd sdks/python +# pip install build && python -m build --sdist +# DataflowRunner: +# python -m pytest -o log_cli=True -o log_level=Info \ +# apache_beam/io/iobase_it_test.py::IOBaseITTest \ +# --test-pipeline-options="--runner=TestDataflowRunner \ +# --project=apache-beam-testing --region=us-central1 \ +# --temp_location=gs://apache-beam-testing-temp/temp \ +# --sdk_location=dist/apache_beam-2.65.0.dev0.tar.gz" + + +class IOBaseITTest(unittest.TestCase): + def setUp(self): + self.test_pipeline = TestPipeline(is_integration_test=True) + self.runner_name = type(self.test_pipeline.runner).__name__ + + def test_unbounded_pcoll_without_gloabl_window(self): + # https://github.com/apache/beam/issues/25598 + + args = self.test_pipeline.get_full_options_as_args(streaming=True, ) + + topic = 'projects/pubsub-public-data/topics/taxirides-realtime' + unique_id = str(uuid.uuid4()) + output_file = f'gs://apache-beam-testing-integration-testing/iobase/test-{unique_id}' # pylint: disable=line-too-long + + p = beam.Pipeline(argv=args) + # Read from Pub/Sub with fixed windowing + lines = ( + p + | "ReadFromPubSub" >> beam.io.ReadFromPubSub(topic=topic) + | "WindowInto" >> beam.WindowInto(FixedWindows(10))) + + # Write to text file + _ = lines | 'WriteToText' >> WriteToText(output_file) + + result = p.run() + result.wait_until_finish(duration=60 * 1000) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + unittest.main() \ No newline at end of file From 403bb72222b0553951265ce208d244ff44224a00 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Mon, 5 May 2025 18:54:31 +0000 Subject: [PATCH 15/19] Add ParquetIO streaming write IT --- sdks/python/apache_beam/io/iobase_it_test.py | 2 +- .../apache_beam/io/parquetio_it_test.py | 39 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/iobase_it_test.py b/sdks/python/apache_beam/io/iobase_it_test.py index f63145ef046a..168d94b41feb 100644 --- a/sdks/python/apache_beam/io/iobase_it_test.py +++ b/sdks/python/apache_beam/io/iobase_it_test.py @@ -69,4 +69,4 @@ def test_unbounded_pcoll_without_gloabl_window(self): if __name__ == "__main__": logging.getLogger().setLevel(logging.INFO) - unittest.main() \ No newline at end of file + unittest.main() diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py index 052b54f3ebfb..0b64902c82c1 100644 --- a/sdks/python/apache_beam/io/parquetio_it_test.py +++ b/sdks/python/apache_beam/io/parquetio_it_test.py @@ -16,13 +16,18 @@ # # pytype: skip-file +import json import logging import string import unittest +import uuid from collections import Counter +from datetime import datetime import pytest +import pytz +import apache_beam as beam from apache_beam import Create from apache_beam import DoFn from apache_beam import FlatMap @@ -37,6 +42,7 @@ from apache_beam.testing.util import BeamAssertException from apache_beam.transforms import CombineGlobally from apache_beam.transforms.combiners import Count +from apache_beam.transforms.periodicsequence import PeriodicImpulse try: import pyarrow as pa @@ -141,6 +147,39 @@ def get_int(self): self._number_index = self._number_index + 1 return i +@unittest.skipIf(pa is None, "PyArrow is not installed.") +class WriteStreamingIT(unittest.TestCase): + def setUp(self): + self.test_pipeline = TestPipeline(is_integration_test=True) + self.runner_name = type(self.test_pipeline.runner).__name__ + super().setUp() + + def test_write_streaming_2_shards_default_shard_name_template( + self, num_shards=2): + + args = self.test_pipeline.get_full_options_as_args(streaming=True, ) + + unique_id = str(uuid.uuid4()) + output_file = f'gs://apache-beam-testing-integration-testing/iobase/test-{unique_id}' # pylint: disable=line-too-long + p = beam.Pipeline(argv=args) + pyschema = pa.schema([('age', pa.int64())]) + + _ = (p + | "generate impulse" >> PeriodicImpulse( + start_timestamp= + datetime(2021, 3, 1, 0, 0, 1, 0,tzinfo=pytz.UTC).timestamp(), + stop_timestamp= + datetime(2021, 3, 1, 0, 0, 20, 0,tzinfo=pytz.UTC).timestamp(), + fire_interval=1) + | "generate data" >> beam.Map(lambda t: {'age': t * 10}) + | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=output_file, + file_name_suffix=".parquet", + num_shards=num_shards, + schema=pyschema) + ) + result = p.run() + result.wait_until_finish(duration=600 * 1000) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 41a1b07cd1bda080143faf0087b11c9f8d18a13b Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Mon, 5 May 2025 19:36:40 +0000 Subject: [PATCH 16/19] formatting fix --- .../apache_beam/io/parquetio_it_test.py | 29 ++++++++++--------- 1 file changed, 16 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py index 0b64902c82c1..b9d784ae7afa 100644 --- a/sdks/python/apache_beam/io/parquetio_it_test.py +++ b/sdks/python/apache_beam/io/parquetio_it_test.py @@ -147,6 +147,7 @@ def get_int(self): self._number_index = self._number_index + 1 return i + @unittest.skipIf(pa is None, "PyArrow is not installed.") class WriteStreamingIT(unittest.TestCase): def setUp(self): @@ -164,23 +165,25 @@ def test_write_streaming_2_shards_default_shard_name_template( p = beam.Pipeline(argv=args) pyschema = pa.schema([('age', pa.int64())]) - _ = (p - | "generate impulse" >> PeriodicImpulse( - start_timestamp= - datetime(2021, 3, 1, 0, 0, 1, 0,tzinfo=pytz.UTC).timestamp(), - stop_timestamp= - datetime(2021, 3, 1, 0, 0, 20, 0,tzinfo=pytz.UTC).timestamp(), - fire_interval=1) - | "generate data" >> beam.Map(lambda t: {'age': t * 10}) - | 'WriteToParquet' >> beam.io.WriteToParquet( - file_path_prefix=output_file, - file_name_suffix=".parquet", - num_shards=num_shards, - schema=pyschema) + _ = ( + p + | "generate impulse" >> PeriodicImpulse( + start_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp(), + stop_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp(), + fire_interval=1) + | "generate data" >> beam.Map(lambda t: {'age': t * 10}) + | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=output_file, + file_name_suffix=".parquet", + num_shards=num_shards, + schema=pyschema) ) result = p.run() result.wait_until_finish(duration=600 * 1000) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) unittest.main() From e685e16c18d31a3ae5400a214afcb1df5ffafa91 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Mon, 5 May 2025 20:02:48 +0000 Subject: [PATCH 17/19] yapf recos --- sdks/python/apache_beam/io/parquetio_it_test.py | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py index b9d784ae7afa..ccb0d41db5dd 100644 --- a/sdks/python/apache_beam/io/parquetio_it_test.py +++ b/sdks/python/apache_beam/io/parquetio_it_test.py @@ -16,7 +16,6 @@ # # pytype: skip-file -import json import logging import string import unittest @@ -168,18 +167,17 @@ def test_write_streaming_2_shards_default_shard_name_template( _ = ( p | "generate impulse" >> PeriodicImpulse( - start_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, - tzinfo=pytz.UTC).timestamp(), - stop_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, - tzinfo=pytz.UTC).timestamp(), - fire_interval=1) + start_timestamp=datetime(2021, 3, 1, 0, 0, 1, 0, + tzinfo=pytz.UTC).timestamp(), + stop_timestamp=datetime(2021, 3, 1, 0, 0, 20, 0, + tzinfo=pytz.UTC).timestamp(), + fire_interval=1) | "generate data" >> beam.Map(lambda t: {'age': t * 10}) | 'WriteToParquet' >> beam.io.WriteToParquet( file_path_prefix=output_file, file_name_suffix=".parquet", num_shards=num_shards, - schema=pyschema) - ) + schema=pyschema)) result = p.run() result.wait_until_finish(duration=600 * 1000) From e28cf60e8ed1dc5d8d1e842b78fdf4177637b254 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 7 May 2025 11:42:01 +0000 Subject: [PATCH 18/19] improve support for user windowing that can be overidden by trigger_frequency param. On GlobalWindow throw an exception if trigger_frequency is not set / 0. --- .../examples/unbounded_sinks/test_write.py | 3 +- sdks/python/apache_beam/io/avroio.py | 2 + sdks/python/apache_beam/io/filebasedsink.py | 2 +- sdks/python/apache_beam/io/iobase.py | 60 +++++-- sdks/python/apache_beam/io/parquetio.py | 1 + .../apache_beam/io/parquetio_it_test.py | 1 + sdks/python/apache_beam/io/parquetio_test.py | 45 +++++ sdks/python/apache_beam/io/textio.py | 26 ++- sdks/python/apache_beam/io/textio_test.py | 166 +++++++++++++++++- sdks/python/apache_beam/io/tfrecordio.py | 2 + 10 files changed, 286 insertions(+), 22 deletions(-) diff --git a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py index 5b52f063b513..99eb30405bf4 100644 --- a/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py +++ b/sdks/python/apache_beam/examples/unbounded_sinks/test_write.py @@ -71,7 +71,8 @@ def run(argv=None, save_main_session=True) -> PipelineResult: file_path_prefix="__output__/ouput_WriteToText", file_name_suffix=".txt", #shard_name_template='-V-SSSSS-of-NNNNN', - num_shards=2, #triggering_frequency=5, + num_shards=2, + triggering_frequency=5, ) _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 78f70d402db4..fac6ba57657b 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -393,6 +393,7 @@ def __init__( Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. + In streaming if not set, the service will write a file per bundle. shard_name_template: A template string containing placeholders for the shard number and shard count. Currently only ``''``, ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and @@ -415,6 +416,7 @@ def __init__( and no longer has an effect. Do not use. triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToAvro transform usable for writing. diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index 447e1c19dd58..ea4b938515d1 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -38,7 +38,7 @@ DEFAULT_SHARD_NAME_TEMPLATE = '-SSSSS-of-NNNNN' DEFAULT_WINDOW_SHARD_NAME_TEMPLATE = '-W-SSSSS-of-NNNNN' -DEFAULT_TRIGGERING_FREQUENCY = 60 +DEFAULT_TRIGGERING_FREQUENCY = 0 __all__ = ['FileBasedSink'] diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index 1b3fdaed1f37..1d9838d645bd 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -1152,16 +1152,33 @@ def expand(self, pcoll): ) else: #unbounded PCollection needes to be written per window if isinstance(pcoll.windowing.windowfn, window.GlobalWindows): + if (self.sink.triggering_frequency is None or + self.sink.triggering_frequency == 0): + raise ValueError( + 'To write a GlobalWindow PCollection, triggering_frequency must' + ' be set and be greater than 0') widowed_pcoll = ( - pcoll + pcoll #TODO GroupIntoBatches and trigger indef per freq | core.WindowInto( window.FixedWindows(self.sink.triggering_frequency), trigger=beam.transforms.trigger.AfterWatermark(), accumulation_mode=beam.transforms.trigger.AccumulationMode. DISCARDING, allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) - else: #keep user windowing - widowed_pcoll = pcoll + else: + #keep user windowing, unless triggering_frequency has been specified + if (self.sink.triggering_frequency is not None and + self.sink.triggering_frequency > 0): + widowed_pcoll = ( + pcoll #TODO GroupIntoBatches and trigger indef per freq + | core.WindowInto( + window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + else: #keep user windowing + widowed_pcoll = pcoll if self.sink.convert_fn is not None: widowed_pcoll = widowed_pcoll | core.ParDo(self.sink.convert_fn) if min_shards == 1: @@ -1214,14 +1231,34 @@ def expand(self, pcoll): | core.GroupByKey() | 'Extract' >> core.FlatMap(lambda x: x[1])) else: #unbounded PCollection needes to be written per window - widowed_pcoll = ( - pcoll - | core.WindowInto( - window.FixedWindows(self.sink.triggering_frequency), - trigger=beam.transforms.trigger.AfterWatermark(), - accumulation_mode=beam.transforms.trigger.AccumulationMode. - DISCARDING, - allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + if isinstance(pcoll.windowing.windowfn, window.GlobalWindows): + if (self.sink.triggering_frequency is None or + self.sink.triggering_frequency == 0): + raise ValueError( + 'To write a GlobalWindow PCollection, triggering_frequency must' + ' be set and be greater than 0') + widowed_pcoll = ( + pcoll #TODO GroupIntoBatches and trigger indef per freq + | core.WindowInto( + window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + else: + #keep user windowing, unless triggering_frequency has been specified + if (self.sink.triggering_frequency is not None and + self.sink.triggering_frequency > 0): + widowed_pcoll = ( + pcoll #TODO GroupIntoBatches and trigger indef per freq + | core.WindowInto( + window.FixedWindows(self.sink.triggering_frequency), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + else: #keep user windowing + widowed_pcoll = pcoll init_result_window_coll = ( widowed_pcoll | 'Pair init' >> core.Map(lambda x: (None, x)) @@ -1352,7 +1389,6 @@ def process( init_result, w=core.DoFn.WindowParam, pane=core.DoFn.PaneInfoParam): - if self.per_key: w_key = "%s_%s" % (w, element[0]) # key else: diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index cee467caa777..89266888792d 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -574,6 +574,7 @@ def __init__( supports specifying MIME types. triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToParquet transform usable for writing. diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py index ccb0d41db5dd..5dd3eac63746 100644 --- a/sdks/python/apache_beam/io/parquetio_it_test.py +++ b/sdks/python/apache_beam/io/parquetio_it_test.py @@ -177,6 +177,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_path_prefix=output_file, file_name_suffix=".parquet", num_shards=num_shards, + triggering_frequency=60, schema=pyschema)) result = p.run() result.wait_until_finish(duration=600 * 1000) diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index ef8e7b8ab940..a65012929f55 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -788,6 +788,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_path_prefix=self.tempdir + "/ouput_WriteToParquet", file_name_suffix=".parquet", num_shards=num_shards, + triggering_frequency=60, schema=pyschema) _ = output2 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) @@ -825,6 +826,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( file_name_suffix=".parquet", shard_name_template=shard_name_template, num_shards=num_shards, + triggering_frequency=60, schema=pyschema) _ = output2 | 'LogElements after WriteToParquet' >> LogElements( prefix='after WriteToParquet ', with_window=True, level=logging.INFO) @@ -896,6 +898,49 @@ def test_write_streaming_2_shards_custom_shard_name_template_5s_window( 10, "expected %d files, but got: %d" % (num_shards, len(file_names))) + def test_write_streaming_undef_shards_default_shard_name_template_windowed_pcoll( # pylint: disable=line-too-long + self): + with TestPipeline() as p: + output = ( + p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(10), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #ParquetIO + pyschema = pa.schema([('age', pa.int64())]) + output2 = output | 'WriteToParquet' >> beam.io.WriteToParquet( + file_path_prefix=self.tempdir + "/ouput_WriteToParquet", + file_name_suffix=".parquet", + num_shards=0, + schema=pyschema) + _ = output2 | 'LogElements after WriteToParquet' >> LogElements( + prefix='after WriteToParquet ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToParquet-[1614556800.0, 1614556805.0)-00000-of-00002.parquet + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.parquet$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToParquet*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertGreaterEqual( + len(file_names), + 1*3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (1*3, len(file_names))) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index c26a01da5d86..7fa31c3d1a00 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -452,7 +452,7 @@ def __init__( max_records_per_shard=None, max_bytes_per_shard=None, skip_if_empty=False, - triggering_frequency=60): + triggering_frequency=None): """Initialize a _TextSink. Args: @@ -469,13 +469,23 @@ def __init__( Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. + In streaming if not set, the service will write a file per bundle. shard_name_template: A template string containing placeholders for - the shard number and shard count. When constructing a filename for a - particular shard number, the upper-case letters 'S' and 'N' are - replaced with the 0-padded shard number and shard count respectively. - This argument can be '' in which case it behaves as if num_shards was - set to 1 and only one file will be generated. The default pattern used - is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template. + the shard number and shard count. Currently only ``''``, + ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and + ``'-V-SSSSS-of-NNNNN'`` are patterns accepted by the service. + When constructing a filename for a particular shard number, the + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'`` for + bounded PCollections and for ``'-W-SSSSS-of-NNNNN'`` unbounded + PCollections. + W is used for windowed shard naming and is replaced with + ``[window.start, window.end)`` + V is used for windowed shard naming and is replaced with + ``[window.start.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S"), + window.end.to_utc_datetime().strftime("%Y-%m-%dT%H-%M-%S")`` coder: Coder used to encode each line. compression_type: Used to handle compressed output files. Typical value is CompressionTypes.AUTO, in which case the final file path's @@ -497,6 +507,8 @@ def __init__( skip_if_empty: Don't write any shards if the PCollection is empty. triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. + Returns: A _TextSink object usable for writing. diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index e88893d5e669..a1183adcc218 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -1984,7 +1984,7 @@ def test_write_streaming_2_shards_default_shard_name_template( file_path_prefix=self.tempdir + "/ouput_WriteToText", file_name_suffix=".txt", num_shards=num_shards, - ) + triggering_frequency=60) _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) @@ -2010,6 +2010,169 @@ def test_write_streaming_2_shards_default_shard_name_template( num_shards, "expected %d files, but got: %d" % (num_shards, len(file_names))) + def test_write_streaming_2_shards_default_shard_name_template_windowed_pcoll( + self, num_shards=2): + with TestPipeline() as p: + output = ( + p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(10), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + num_shards=num_shards, + ) + _ = output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertEqual( + len(file_names), + num_shards*3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (num_shards*3, len(file_names))) + + def test_write_streaming_undef_shards_default_shard_name_template_windowed_pcoll( # pylint: disable=line-too-long + self): + with TestPipeline() as p: + output = ( + p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(10), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + num_shards=0, + ) + _ = output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertGreaterEqual( + len(file_names), + 1*3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (1*3, len(file_names))) + + def test_write_streaming_undef_shards_default_shard_name_template_windowed_pcoll_and_trig_freq( # pylint: disable=line-too-long + self): + with TestPipeline() as p: + output = ( + p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(60), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + num_shards=0, + triggering_frequency=10, + ) + _ = output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertGreaterEqual( + len(file_names), + 1*3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (1*3, len(file_names))) + + def test_write_streaming_undef_shards_default_shard_name_template_global_window_pcoll( # pylint: disable=line-too-long + self): + with TestPipeline() as p: + output = (p | GenerateEvent.sample_data()) + #TextIO + output2 = output | 'TextIO WriteToText' >> beam.io.WriteToText( + file_path_prefix=self.tempdir + "/ouput_WriteToText", + file_name_suffix=".txt", + num_shards=0, #0 means undef nb of shards, same as omitted/default + triggering_frequency=60, + ) + _ = output2 | 'LogElements after WriteToText' >> LogElements( + prefix='after WriteToText ', with_window=True, level=logging.INFO) + + # Regex to match the expected windowed file pattern + # Example: + # ouput_WriteToText-[1614556800.0, 1614556805.0)-00000-of-00002.txt + # It captures: window_interval, shard_num, total_shards + pattern_string = ( + r'.*-\[(?P[\d\.]+), ' + r'(?P[\d\.]+|Infinity)\)-' + r'(?P\d{5})-of-(?P\d{5})\.txt$') + pattern = re.compile(pattern_string) + file_names = [] + for file_name in glob.glob(self.tempdir + '/ouput_WriteToText*'): + match = pattern.match(file_name) + self.assertIsNotNone( + match, f"File name {file_name} did not match expected pattern.") + if match: + file_names.append(file_name) + print("Found files matching expected pattern:", file_names) + self.assertGreaterEqual( + len(file_names), + 1*3, #25s of data covered by 3 10s windows + "expected %d files, but got: %d" % (1*3, len(file_names))) + def test_write_streaming_2_shards_custom_shard_name_template( self, num_shards=2, shard_name_template='-V-SSSSS-of-NNNNN'): with TestPipeline() as p: @@ -2020,6 +2183,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( file_name_suffix=".txt", shard_name_template=shard_name_template, num_shards=num_shards, + triggering_frequency=60, ) _ = output2 | 'LogElements after WriteToText' >> LogElements( prefix='after WriteToText ', with_window=True, level=logging.INFO) diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py index 57ca00e38c05..20617663e95c 100644 --- a/sdks/python/apache_beam/io/tfrecordio.py +++ b/sdks/python/apache_beam/io/tfrecordio.py @@ -329,6 +329,7 @@ def __init__( file_name_suffix: Suffix for the files written. num_shards: The number of files (shards) used for output. If not set, the default value will be used. + In streaming if not set, the service will write a file per bundle. shard_name_template: A template string containing placeholders for the shard number and shard count. Currently only ``''``, ``'-SSSSS-of-NNNNN'``, ``'-W-SSSSS-of-NNNNN'`` and @@ -350,6 +351,7 @@ def __init__( be used to detect the compression. triggering_frequency: (int) Every triggering_frequency duration, a window will be triggered and all bundles in the window will be written. + If set it overrides user windowing. Mandatory for GlobalWindow. Returns: A WriteToTFRecord transform object. From 70d4c2412075f3a7dc2ff8359d1ac7e4c4b22ce8 Mon Sep 17 00:00:00 2001 From: Razvan Culea Date: Wed, 7 May 2025 12:35:37 +0000 Subject: [PATCH 19/19] fix unittest needing either user window or triggering_frequency --- sdks/python/apache_beam/io/avroio_test.py | 11 ++++++++++- sdks/python/apache_beam/io/tfrecordio_test.py | 7 +++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 20d4f0725bcc..efbdef1ca44a 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -755,7 +755,15 @@ def tearDown(self): def test_write_streaming_2_shards_default_shard_name_template( self, num_shards=2): with TestPipeline() as p: - output = (p | GenerateEvent.sample_data()) + output = ( + p + | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(60), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0))) #AvroIO avroschema = { 'name': 'dummy', # your supposed to be file name with .avro extension @@ -812,6 +820,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( file_name_suffix=".avro", shard_name_template=shard_name_template, num_shards=num_shards, + triggering_frequency=60, schema=avroschema) _ = output2 | 'LogElements after WriteToAvro' >> LogElements( prefix='after WriteToAvro ', with_window=True, level=logging.INFO) diff --git a/sdks/python/apache_beam/io/tfrecordio_test.py b/sdks/python/apache_beam/io/tfrecordio_test.py index 5875e4511809..6522ade36d80 100644 --- a/sdks/python/apache_beam/io/tfrecordio_test.py +++ b/sdks/python/apache_beam/io/tfrecordio_test.py @@ -691,6 +691,12 @@ def test_write_streaming_2_shards_default_shard_name_template( output = ( p | GenerateEvent.sample_data() + | 'User windowing' >> beam.transforms.core.WindowInto( + beam.transforms.window.FixedWindows(60), + trigger=beam.transforms.trigger.AfterWatermark(), + accumulation_mode=beam.transforms.trigger.AccumulationMode. + DISCARDING, + allowed_lateness=beam.utils.timestamp.Duration(seconds=0)) | "encode" >> beam.Map(lambda s: json.dumps(s).encode('utf-8'))) #TFrecordIO output2 = output | 'WriteToTFRecord' >> beam.io.WriteToTFRecord( @@ -736,6 +742,7 @@ def test_write_streaming_2_shards_custom_shard_name_template( file_name_suffix=".tfrecord", shard_name_template=shard_name_template, num_shards=num_shards, + triggering_frequency=60, ) _ = output2 | 'LogElements after WriteToTFRecord' >> LogElements( prefix='after WriteToTFRecord ', with_window=True, level=logging.INFO)