[
https://issues.apache.org/jira/browse/BEAM-5993?focusedWorklogId=177132&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-177132
]
ASF GitHub Bot logged work on BEAM-5993:
----------------------------------------
Author: ASF GitHub Bot
Created on: 19/Dec/18 18:51
Start Date: 19/Dec/18 18:51
Worklog Time Spent: 10m
Work Description: pabloem closed pull request #7020: BEAM-5993 Create
SideInput Load test
URL: https://github.com/apache/beam/pull/7020
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git a/sdks/python/apache_beam/testing/load_tests/sideinput_test.py
b/sdks/python/apache_beam/testing/load_tests/sideinput_test.py
new file mode 100644
index 000000000000..435f44c02eb4
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/sideinput_test.py
@@ -0,0 +1,203 @@
+#
+# 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.
+#
+"""
+This is SideInput load test with Synthetic Source. Besides of the standard
+input options there are additional options:
+* number_of_counter_operations - number of pardo operations
+* project (optional) - the gcp project in case of saving
+metrics in Big Query (in case of Dataflow Runner
+it is required to specify project of runner),
+* metrics_table (optional) - name of BigQuery table where metrics
+will be stored,
+in case of lack of any of both options metrics won't be saved
+* metrics_dataset (optional) - name of BigQuery dataset where metrics
+will be stored,
+in case of lack of all three options metrics won't be saved
+* input_options - options for Synthetic Sources.
+
+To run test on DirectRunner
+
+python setup.py nosetests \
+ --project=big-query-project
+ --metrics_dataset=python_load_tests
+ --metrics_table=side_input
+ --test-pipeline-options="
+ --number_of_counter_operations=1000
+ --input_options='{
+ \"num_records\": 300,
+ \"key_size\": 5,
+ \"value_size\":15,
+ \"bundle_size_distribution_type\": \"const\",
+ \"bundle_size_distribution_param\": 1,
+ \"force_initial_num_bundles\": 0
+ }'
+ " \
+ --tests apache_beam.testing.load_tests.sideinput_test
+
+To run test on other runner (ex. Dataflow):
+
+python setup.py nosetests \
+ --test-pipeline-options="
+ --runner=TestDataflowRunner
+ --project=...
+ --metrics_dataset=python_load_tests
+ --metrics_table=side_input
+ --staging_location=gs://...
+ --temp_location=gs://...
+ --sdk_location=./dist/apache-beam-x.x.x.dev0.tar.gz
+ --number_of_counter_operations=1000
+ --input_options='{
+ \"num_records\": 1,
+ \"key_size\": 1,
+ \"value_size\":1,
+ \"bundle_size_distribution_type\": \"const\",
+ \"bundle_size_distribution_param\": 1,
+ \"force_initial_num_bundles\": 0
+ }'
+ " \
+ --tests apache_beam.testing.load_tests.sideinput_test
+
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import os
+import unittest
+
+import apache_beam as beam
+from apache_beam.pvalue import AsIter
+from apache_beam.testing import synthetic_pipeline
+from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime
+from apache_beam.testing.load_tests.load_test_metrics_utils import
MetricsMonitor
+from apache_beam.testing.test_pipeline import TestPipeline
+
+load_test_enabled = False
+if os.environ.get('LOAD_TEST_ENABLED') == 'true':
+ load_test_enabled = True
+
+RUNTIME_LABEL = 'runtime'
+
+
[email protected](not load_test_enabled, 'Enabled only for phase triggering.')
+class SideInputTest(unittest.TestCase):
+ def _parseTestPipelineOptions(self):
+ return {
+ 'numRecords': self.inputOptions.get('num_records'),
+ 'keySizeBytes': self.inputOptions.get('key_size'),
+ 'valueSizeBytes': self.inputOptions.get('value_size'),
+ 'bundleSizeDistribution': {
+ 'type': self.inputOptions.get(
+ 'bundle_size_distribution_type', 'const'
+ ),
+ 'param': self.inputOptions.get('bundle_size_distribution_param', 0)
+ },
+ 'forceNumInitialBundles': self.inputOptions.get(
+ 'force_initial_num_bundles', 0
+ )
+ }
+
+ def _getSideInput(self):
+ side_input = self._parseTestPipelineOptions()
+ side_input['numRecords'] = side_input['numRecords']
+ side_input['keySizeBytes'] = side_input['keySizeBytes']
+ side_input['valueSizeBytes'] = side_input['valueSizeBytes']
+ return side_input
+
+ def _getPerElementDelaySec(self):
+ return self.syntheticStepOptions.get('per_element_delay_sec', 0)
+
+ def _getPerBundleDelaySec(self):
+ return self.syntheticStepOptions.get('per_bundle_delay_sec', 0)
+
+ def _getOutputRecordsPerInputRecords(self):
+ return self.syntheticStepOptions.get('output_records_per_input_records', 0)
+
+ def setUp(self):
+ self.pipeline = TestPipeline()
+ self.inputOptions = json.loads(self.pipeline.get_option('input_options'))
+ self.iterations = self.pipeline.get_option('number_of_counter_operations')
+ if self.iterations is None:
+ self.iterations = 1
+ self.iterations = int(self.iterations)
+
+ metrics_project_id = self.pipeline.get_option('project')
+ self.metrics_namespace = self.pipeline.get_option('metrics_table')
+ if not self.metrics_namespace:
+ self.metrics_namespace = self.__class__.__name__
+ metrics_dataset = self.pipeline.get_option('metrics_dataset')
+ self.metrics_monitor = None
+ check = metrics_project_id and self.metrics_namespace and metrics_dataset \
+ is not None
+ if check:
+ measured_values = [
+ {'name': RUNTIME_LABEL, 'type': 'FLOAT', 'mode': 'REQUIRED'},
+ ]
+ self.metrics_monitor = MetricsMonitor(
+ project_name=metrics_project_id,
+ table=self.metrics_namespace,
+ dataset=metrics_dataset,
+ schema_map=measured_values
+ )
+ else:
+ logging.error('One or more of parameters for collecting metrics '
+ 'are empty. Metrics will not be collected')
+
+ def testSideInput(self):
+ def join_fn(element, side_input, iterations):
+ list = []
+ for i in range(iterations):
+ for key, value in side_input:
+ if i == iterations - 1:
+ list.append({key: element[1]+value})
+ yield list
+
+ with self.pipeline as p:
+ main_input = (p
+ | "Read pcoll 1" >> beam.io.Read(
+ synthetic_pipeline.SyntheticSource(
+ self._parseTestPipelineOptions()))
+ | 'Measure time: Start pcoll 1' >> beam.ParDo(
+ MeasureTime(self.metrics_namespace))
+ )
+
+ side_input = (p
+ | "Read pcoll 2" >> beam.io.Read(
+ synthetic_pipeline.SyntheticSource(
+ self._getSideInput()))
+ | 'Measure time: Start pcoll 2' >> beam.ParDo(
+ MeasureTime(self.metrics_namespace))
+ )
+ # pylint: disable=expression-not-assigned
+ (main_input
+ | "Merge" >> beam.ParDo(
+ join_fn,
+ AsIter(side_input),
+ self.iterations)
+ | 'Measure time' >> beam.ParDo(MeasureTime(self.metrics_namespace))
+ )
+
+ result = p.run()
+ result.wait_until_finish()
+
+ if self.metrics_monitor is not None:
+ self.metrics_monitor.send_metrics(result)
+
+ if __name__ == '__main__':
+ logging.getLogger().setLevel(logging.DEBUG)
+ unittest.main()
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 177132)
Time Spent: 1h 40m (was: 1.5h)
> Create SideInput Load test
> --------------------------
>
> Key: BEAM-5993
> URL: https://issues.apache.org/jira/browse/BEAM-5993
> Project: Beam
> Issue Type: Sub-task
> Components: testing
> Reporter: Kasia Kucharczyk
> Assignee: Kasia Kucharczyk
> Priority: Major
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)