You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/11/07 16:44:50 UTC

[GitHub] [beam] ryanthompson591 commented on a diff in pull request #23951: Python TextIO Performance Test

ryanthompson591 commented on code in PR #23951:
URL: https://github.com/apache/beam/pull/23951#discussion_r1015627562


##########
.test-infra/jenkins/job_PerformanceTests_BigQueryIO_Python.groovy:
##########
@@ -90,7 +90,7 @@ PhraseTriggeringPostCommitBuilder.postCommitJob(
       executeJob(delegate, bqio_read_test)
     }
 
-CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Read_Python', 'H 15 * * *', this) {
+CronJobBuilder.cronJob('beam_PerformanceTests_BiqQueryIO_Read_Python', 'H H * * *', this) {

Review Comment:
   what does H H mean?



##########
.test-infra/metrics/grafana/dashboards/perftests_metrics/Python_IO_IT_Tests_Dataflow.json:
##########
@@ -482,6 +482,128 @@
         "align": false,
         "alignLevel": null
       }
+    },

Review Comment:
   Feel free to add another reviewer for these graphana changes as I'm not really familiar enough to give good feedback.



##########
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).')
+
+
+@typehints.with_output_types(bytes)
+@typehints.with_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

Review Comment:
   this output length is a little hard to read.  Is this machine read somewhere? If not maybe just describe it?



##########
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).')
+
+
+@typehints.with_output_types(bytes)

Review Comment:
   why is this base64 encoded data bytes and not string as output type?



##########
.test-infra/jenkins/job_PerformanceTests_SpannerIO_Python.groovy:
##########
@@ -92,7 +92,7 @@ PhraseTriggeringPostCommitBuilder.postCommitJob(
       executeJob(delegate, spannerio_read_test_2gb)
     }
 
-CronJobBuilder.cronJob('beam_PerformanceTests_SpannerIO_Read_2GB_Python', 'H 15 * * *', this) {
+CronJobBuilder.cronJob('beam_PerformanceTests_SpannerIO_Read_2GB_Python', 'H H * * *', this) {

Review Comment:
   On line 108 you added a batch test, does that mean this one is streaming?



##########
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).')
+
+
+@typehints.with_output_types(bytes)
+@typehints.with_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)

Review Comment:
   AFAICT this test is only implemented in one place and need_cleanup is always false, why not just remove this variable?



##########
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(

Review Comment:
   IMO, things that you will not configure can be hard coded.  Will dataset_size always be the same?
   
   I'll leave keeping this or removing up to you.



##########
sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py:
##########
@@ -385,7 +383,13 @@ def _prepare_runtime_metrics(self, distributions):
     return runtime_in_s
 
 
-class ConsoleMetricsPublisher(object):
+class MetricsPublisher:

Review Comment:
   what does this base class gain for us?



##########
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).')
+
+
+@typehints.with_output_types(bytes)
+@typehints.with_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()

Review Comment:
   Why do we need Impulse here?



##########
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).')
+
+
+@typehints.with_output_types(bytes)
+@typehints.with_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'm worried about a race case here. Since the folder creation is parallel to the writing, I'm worried it could happen that the folder creation could happen too late.
   
   What do you think of this change.
   ```
   
   after_folder_creation = (
           self.pipeline
           | beam.Impulse()
           | beam.ParDo(CreateFolderFn(self.output_folder)))
   
     # write to text
       _ = (
           after_folder_creation ### Make sure this happens after the creation of a folder.
           | 'Produce rows' >> Read(
               SyntheticSource(self.parse_synthetic_source_options()))
   
   ```



##########
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).')
+
+
+@typehints.with_output_types(bytes)
+@typehints.with_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)))
+
+    # write to text
+    _ = (
+        self.pipeline
+        | 'Produce rows' >> Read(
+            SyntheticSource(self.parse_synthetic_source_options()))
+        | 'Count records' >> beam.ParDo(CountMessages(self.metrics_namespace))
+        | 'Format' >> beam.ParDo(SyntheticRecordToStrFn())

Review Comment:
   Why do we need to format these records?



-- 
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: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org