Abacn commented on code in PR #23951: URL: https://github.com/apache/beam/pull/23951#discussion_r1015716497
########## sdks/python/apache_beam/io/filebasedio_perf_test.py: ########## @@ -0,0 +1,185 @@ +# +# 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. +# + +"""Performance tests for file based io connectors.""" + +import logging +import sys +import uuid +from typing import Tuple + +import apache_beam as beam +from apache_beam import typehints +from apache_beam.io.filesystems import FileSystems +from apache_beam.io.iobase import Read +from apache_beam.io.textio import ReadFromText +from apache_beam.io.textio import WriteToText +from apache_beam.testing.load_tests.load_test import LoadTest +from apache_beam.testing.load_tests.load_test import LoadTestOptions +from apache_beam.testing.load_tests.load_test_metrics_utils import CountMessages +from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime +from apache_beam.testing.synthetic_pipeline import SyntheticSource +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + +WRITE_NAMESPACE = 'write' +READ_NAMESPACE = 'read' + +_LOGGER = logging.getLogger(__name__) + + +class FileBasedIOTestOptions(LoadTestOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--test_class', required=True, help='Test class to run.') + parser.add_argument( + '--filename_prefix', + required=True, + help='Destination prefix for files generated by the test.') + parser.add_argument( + '--compression_type', + default='auto', + help='File compression type for writing and reading test files.') + parser.add_argument( + '--number_of_shards', + type=int, + default=0, + help='Number of files this test will create during the write phase.') + parser.add_argument( + '--dataset_size', + type=int, + help='Size of data saved on the target filesystem (bytes).') + + [email protected]_output_types(bytes) [email protected]_input_types(Tuple[bytes, bytes]) +class SyntheticRecordToStrFn(beam.DoFn): + """ + A DoFn that convert key-value bytes from synthetic source to string record. + + Output length = 4(ceil[len(key)/3] + ceil[len(value)/3]) + 1 + """ + def process(self, element): + import base64 + yield base64.b64encode(element[0]) + b' ' + base64.b64encode(element[1]) + + +class CreateFolderFn(beam.DoFn): + """Create folder at pipeline runtime.""" + def __init__(self, folder): + self.folder = folder + + def process(self, element): + from apache_beam.io.filesystems import FileSystems # pylint: disable=reimported + filesystem = FileSystems.get_filesystem(self.folder) + if filesystem.has_dirs() and not filesystem.exists(self.folder): + filesystem.mkdirs(self.folder) + + +class TextIOPerfTest: + def run(self): + write_test = _TextIOWritePerfTest(need_cleanup=False) + read_test = _TextIOReadPerfTest(input_folder=write_test.output_folder) + write_test.run() + read_test.run() + + +class _TextIOWritePerfTest(LoadTest): + def __init__(self, need_cleanup=True): + super().__init__(WRITE_NAMESPACE) + self.need_cleanup = need_cleanup + self.test_options = self.pipeline.get_pipeline_options().view_as( + FileBasedIOTestOptions) + self.output_folder = FileSystems.join( + self.test_options.filename_prefix, str(uuid.uuid4())) + + def test(self): + # first makedir if needed + _ = ( + self.pipeline + | beam.Impulse() + | beam.ParDo(CreateFolderFn(self.output_folder))) Review Comment: I have to create the folder at pipeline runtime because the tests should work fine when only pipeline runner has access to the filesystem, and the submitter may not have. For GCS filesystem it is fine. mkdirs is a non-op as GCS is a flat structure. Here it is aimed to make the test run in both nested filesystem (like local filesystem) and flat filesystem (like gcs). ~The change would be the same because the Read transform essentially expands from pbegin (it also uses Impulse which creates a PCollection from pbegin.pipeline). The pipeline graph will remain the same.~Tried that Read() must follow pbegin because it uses Impulse() which asserts the upstream is pbegin. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
