[ 
https://issues.apache.org/jira/browse/BEAM-5778?focusedWorklogId=162541&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-162541
 ]

ASF GitHub Bot logged work on BEAM-5778:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 05/Nov/18 14:58
            Start Date: 05/Nov/18 14:58
    Worklog Time Spent: 10m 
      Work Description: kkucharc closed pull request #6941: [BEAM-5778] Add 
integrations of Metrics API to Big Query for SyntheticSources load tests in 
Python SDK
URL: https://github.com/apache/beam/pull/6941
 
 
   

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/__init__.py 
b/sdks/python/apache_beam/testing/load_tests/__init__.py
new file mode 100644
index 00000000000..cce3acad34a
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/__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/testing/load_tests/co_group_by_key_test.py 
b/sdks/python/apache_beam/testing/load_tests/co_group_by_key_test.py
new file mode 100644
index 00000000000..5decc2bdf65
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/co_group_by_key_test.py
@@ -0,0 +1,162 @@
+#
+# 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.
+#
+"""
+To run test on DirectRunner
+
+python setup.py nosetests \
+    --test-pipeline-options="
+      --metrics_project_id=big-query-project
+      --input_options='{
+        \"num_records\": 1000,
+        \"key_size\": 5,
+        \"value_size\":15,
+        \"bundle_size_distribution_type\": \"const\",
+        \"bundle_size_distribution_param\": 1,
+        \"force_initial_num_bundles\":0}'
+        --co_input_options='{
+        \"num_records\": 1000,
+        \"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.co_group_by_it_test
+
+To run test on other runner (ex. Dataflow):
+
+python setup.py nosetests \
+    --test-pipeline-options="
+        --runner=TestDataflowRunner
+        --project=...
+        --staging_location=gs://...
+        --temp_location=gs://...
+        --sdk_location=./dist/apache-beam-x.x.x.dev0.tar.gz
+        --metrics_project_id=big-query-project
+        --input_options='{
+        \"num_records\": 1000,
+        \"key_size\": 5,
+        \"value_size\":15,
+        \"bundle_size_distribution_type\": \"const\",
+        \"bundle_size_distribution_param\": 1,
+        \"force_initial_num_bundles\":0
+        }'
+        --co_input_options='{
+        \"num_records\": 1000,
+        \"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.co_group_by_it_test
+
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing import synthetic_pipeline
+from apache_beam.testing.load_tests.load_test_metrics_utils import 
BigQueryMetricsCollector
+from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime
+from apache_beam.testing.test_pipeline import TestPipeline
+
+INPUT_TAG = 'pc1'
+CO_INPUT_TAG = 'pc2'
+NAMESPACE = 'co_gbk'
+RUNTIME_LABEL = 'runtime'
+
+
+class CoGroupByKeyTest(unittest.TestCase):
+
+  def parseTestPipelineOptions(self, options):
+    return {
+        'numRecords': options.get('num_records'),
+        'keySizeBytes': options.get('key_size'),
+        'valueSizeBytes': options.get('value_size'),
+        'bundleSizeDistribution': {
+            'type': options.get(
+                'bundle_size_distribution_type', 'const'
+            ),
+            'param': options.get('bundle_size_distribution_param', 0)
+        },
+        'forceNumInitialBundles': options.get(
+            'force_initial_num_bundles', 0
+        )
+    }
+
+  def setUp(self):
+    self.pipeline = TestPipeline(is_integration_test=True)
+    self.inputOptions = json.loads(self.pipeline.get_option('input_options'))
+    self.coInputOptions = json.loads(
+        self.pipeline.get_option('co_input_options'))
+
+    metrics_project_id = self.pipeline.get_option('metrics_project_id')
+    self.bigQuery = None
+    if metrics_project_id is not None:
+      schema = [{'name': RUNTIME_LABEL, 'type': 'FLOAT', 'mode': 'REQUIRED'}]
+      self.bigQuery = BigQueryMetricsCollector(
+          metrics_project_id,
+          NAMESPACE,
+          schema
+      )
+
+  class _Ungroup(beam.DoFn):
+    def process(self, element):
+      values = element[1]
+      inputs = values.get(INPUT_TAG)
+      co_inputs = values.get(CO_INPUT_TAG)
+      for i in inputs:
+        yield i
+      for i in co_inputs:
+        yield i
+
+  def testCoGroupByKey(self):
+    with self.pipeline as p:
+      pc1 = (p
+             | 'Read ' + INPUT_TAG >> beam.io.Read(
+                 synthetic_pipeline.SyntheticSource(
+                     self.parseTestPipelineOptions(self.inputOptions)))
+             | 'Make ' + INPUT_TAG + ' iterable' >> beam.Map(lambda x: (x, x))
+            )
+
+      pc2 = (p
+             | 'Read ' + CO_INPUT_TAG >> beam.io.Read(
+                 synthetic_pipeline.SyntheticSource(
+                     self.parseTestPipelineOptions(self.coInputOptions)))
+             | 'Make ' + CO_INPUT_TAG + ' iterable' >> beam.Map(
+                 lambda x: (x, x))
+            )
+      # pylint: disable=expression-not-assigned
+      ({INPUT_TAG: pc1, CO_INPUT_TAG: pc2}
+       | 'CoGroupByKey: ' >> beam.CoGroupByKey()
+       | 'Consume Joined Collections' >> beam.ParDo(self._Ungroup())
+       | 'Measure time' >> beam.ParDo(MeasureTime(NAMESPACE))
+      )
+
+      result = p.run()
+      result.wait_until_finish()
+      if self.bigQuery is not None:
+        self.bigQuery.save_metrics(result)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()
diff --git a/sdks/python/apache_beam/testing/load_tests/combine_test.py 
b/sdks/python/apache_beam/testing/load_tests/combine_test.py
new file mode 100644
index 00000000000..50576453f99
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/combine_test.py
@@ -0,0 +1,126 @@
+#
+# 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.
+#
+"""
+To run test on DirectRunner
+
+python setup.py nosetests \
+    --test-pipeline-options="
+    --metrics_project_id=big-query-project
+    --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.combine_test
+
+To run test on other runner (ex. Dataflow):
+
+python setup.py nosetests \
+    --test-pipeline-options="
+        --runner=TestDataflowRunner
+        --project=...
+        --staging_location=gs://...
+        --temp_location=gs://...
+        --sdk_location=./dist/apache-beam-x.x.x.dev0.tar.gz
+        --metrics_project_id=big-query-project
+        --input_options='{
+        \"num_records\": 1000,
+        \"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.combine_test
+
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing import synthetic_pipeline
+from apache_beam.testing.load_tests.load_test_metrics_utils import 
BigQueryMetricsCollector
+from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime
+from apache_beam.testing.test_pipeline import TestPipeline
+
+NAMESPACE = 'combine'
+RUNTIME_LABEL = 'runtime'
+
+
+class CombineTest(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 setUp(self):
+    self.pipeline = TestPipeline(is_integration_test=True)
+    self.inputOptions = json.loads(self.pipeline.get_option('input_options'))
+
+    metrics_project_id = self.pipeline.get_option('metrics_project_id')
+    self.bigQuery = None
+    if metrics_project_id is not None:
+      schema = [{'name': RUNTIME_LABEL, 'type': 'FLOAT', 'mode': 'REQUIRED'}]
+      self.bigQuery = BigQueryMetricsCollector(
+          metrics_project_id,
+          NAMESPACE,
+          schema
+      )
+
+  class _GetElement(beam.DoFn):
+    def process(self, element):
+      yield element
+
+  def testCombineGlobally(self):
+    with self.pipeline as p:
+      # pylint: disable=expression-not-assigned
+      (p
+       | beam.io.Read(synthetic_pipeline.SyntheticSource(
+           self.parseTestPipelineOptions()))
+       | 'Measure time' >> beam.ParDo(MeasureTime(NAMESPACE))
+       | 'Combine with Top' >> beam.CombineGlobally(
+           beam.combiners.TopCombineFn(1000))
+       | 'Consume' >> beam.ParDo(self._GetElement())
+      )
+
+      result = p.run()
+      result.wait_until_finish()
+      if self.bigQuery is not None:
+        self.bigQuery.save_metrics(result)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.DEBUG)
+  unittest.main()
diff --git a/sdks/python/apache_beam/testing/load_tests/group_by_key_test.py 
b/sdks/python/apache_beam/testing/load_tests/group_by_key_test.py
new file mode 100644
index 00000000000..f252bee3c79
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/group_by_key_test.py
@@ -0,0 +1,122 @@
+#
+# 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.
+#
+"""
+To run test on DirectRunner
+
+python setup.py nosetests \
+    --test-pipeline-options="
+    --metrics_project_id=big-query-project
+    --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.group_by_key_test
+
+To run test on other runner (ex. Dataflow):
+
+python setup.py nosetests \
+    --test-pipeline-options="
+        --runner=TestDataflowRunner
+        --project=...
+        --staging_location=gs://...
+        --temp_location=gs://...
+        --sdk_location=./dist/apache-beam-x.x.x.dev0.tar.gz
+        --metrics_project_id=big-query-project
+        --input_options='{
+        \"num_records\": 1000,
+        \"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.group_by_key_test
+
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing import synthetic_pipeline
+from apache_beam.testing.load_tests.load_test_metrics_utils import 
BigQueryMetricsCollector
+from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime
+from apache_beam.testing.test_pipeline import TestPipeline
+
+NAMESPACE = 'gbk'
+RUNTIME_LABEL = 'runtime'
+
+
+class GroupByKeyTest(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 setUp(self):
+    self.pipeline = TestPipeline(is_integration_test=True)
+    self.inputOptions = json.loads(self.pipeline.get_option('input_options'))
+
+    metrics_project_id = self.pipeline.get_option('metrics_project_id')
+    self.bigQuery = None
+    if metrics_project_id is not None:
+      schema = [{'name': RUNTIME_LABEL, 'type': 'FLOAT', 'mode': 'REQUIRED'}]
+      self.bigQuery = BigQueryMetricsCollector(
+          metrics_project_id,
+          NAMESPACE,
+          schema
+      )
+
+  def testGroupByKey(self):
+    with self.pipeline as p:
+      # pylint: disable=expression-not-assigned
+      (p
+       | beam.io.Read(synthetic_pipeline.SyntheticSource(
+           self.parseTestPipelineOptions()))
+       | 'Measure time' >> beam.ParDo(MeasureTime(NAMESPACE))
+       | 'GroupByKey' >> beam.GroupByKey()
+       | 'Ungroup' >> beam.FlatMap(
+           lambda elm: [(elm[0], v) for v in elm[1]])
+      )
+
+      result = p.run()
+      result.wait_until_finish()
+      if self.bigQuery is not None:
+        self.bigQuery.save_metrics(result)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.DEBUG)
+  unittest.main()
diff --git 
a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py 
b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py
new file mode 100644
index 00000000000..9a7b792428c
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py
@@ -0,0 +1,162 @@
+#
+# 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.
+#
+
+"""
+Utility functions used for integrating Metrics API into load tests pipelines.
+"""
+
+from __future__ import absolute_import
+
+import logging
+import time
+
+from google.cloud import bigquery
+from google.cloud.bigquery.schema import SchemaField
+
+import apache_beam as beam
+from apache_beam.metrics import Metrics
+
+START_TIME_LABEL = 'runtime_start'
+END_TIME_LABEL = 'runtime_end'
+RUNTIME_LABEL = 'runtime'
+SUBMIT_TIMESTAMP_LABEL = 'submit_timestamp'
+LOAD_TEST_DATASET_NAME = 'python_load_tests'
+
+
+def _get_schema_field(schema_field):
+  return SchemaField(
+      name=schema_field['name'],
+      field_type=schema_field['type'],
+      mode=schema_field['mode'])
+
+
+class BigQueryMetricsCollector(object):
+  def __init__(self, project_name, namespace, schema_map):
+    self._namespace = namespace
+    bq_client = bigquery.Client(project=project_name)
+    bq_dataset = bq_client.dataset(LOAD_TEST_DATASET_NAME)
+    if not bq_dataset.exists():
+      raise ValueError(
+          'Dataset {} does not exist in your project. '
+          'You have to create table first.'
+          .format(namespace))
+
+    schemas = [{'name': SUBMIT_TIMESTAMP_LABEL,
+                'type': 'TIMESTAMP',
+                'mode': 'REQUIRED'}] + schema_map
+
+    self._schema_names = [schema['name'] for schema in schemas]
+
+    bq_schemas = [_get_schema_field(schema) for schema in schemas]
+
+    self._bq_table = bq_dataset.table(namespace, bq_schemas)
+    if not self._bq_table.exists():
+      self._bq_table.create()
+
+  def save_metrics(self, result):
+    metrics = result.metrics().query()
+    counters = metrics['counters']
+    counters_list = []
+    if len(counters) > 0:
+      counters_list = self._prepare_counter_metrics(counters)
+
+    distributions = metrics['distributions']
+    dist_list = []
+    if len(distributions) > 0:
+      dist_list = self._prepare_runtime_metrics(distributions)
+
+    timestamp = {'label': SUBMIT_TIMESTAMP_LABEL, 'value': time.time()}
+
+    insert_list = [timestamp] + dist_list + counters_list
+    rows_tuple = tuple(self._match_inserts_by_schema(insert_list))
+
+    self.insert_metrics(rows_tuple)
+
+  def _prepare_counter_metrics(self, counters):
+    for counter in counters:
+      logging.info("Counter:  %s", counter)
+    counters_list = []
+    for counter in counters:
+      counter_commited = counter.committed
+      counter_label = str(counter.key.metric.name)
+      counters_list.append({'label': counter_label, 'value': counter_commited})
+
+    return counters_list
+
+  # prepares distributions of start and end metrics to show test runtime
+  def _prepare_runtime_metrics(self, distributions):
+    dist_pivot = {}
+    for dist in distributions:
+      logging.info("Distribution: %s", dist)
+      dist_pivot.update(self._get_start_end_time(dist))
+
+    runtime_in_s = dist_pivot[END_TIME_LABEL] - dist_pivot[START_TIME_LABEL]
+    runtime_in_s = float(runtime_in_s)
+    return [{'label': RUNTIME_LABEL, 'value': runtime_in_s}]
+
+  def _match_inserts_by_schema(self, insert_list):
+    for name in self._schema_names:
+      yield self._get_element_by_schema(name, insert_list)
+
+  def _get_element_by_schema(self, schema_name, list):
+    for metric in list:
+      if metric['label'] == schema_name:
+        return metric['value']
+    return None
+
+  def insert_metrics(self, rows_tuple):
+    job = self._bq_table.insert_data(rows=[rows_tuple])
+    if len(job) > 0 and len(job[0]['errors']) > 0:
+      for err in job[0]['errors']:
+        raise ValueError(err['message'])
+
+  def _get_start_end_time(self, distribution):
+    if distribution.key.metric.name == START_TIME_LABEL:
+      return {distribution.key.metric.name: distribution.committed.min}
+    elif distribution.key.metric.name == END_TIME_LABEL:
+      return {distribution.key.metric.name: distribution.committed.max}
+
+
+class MeasureTime(beam.DoFn):
+  def __init__(self, namespace):
+    self.namespace = namespace
+    self.runtime_start = Metrics.distribution(self.namespace, START_TIME_LABEL)
+    self.runtime_end = Metrics.distribution(self.namespace, END_TIME_LABEL)
+
+  def start_bundle(self):
+    self.runtime_start.update(time.time())
+
+  def finish_bundle(self):
+    self.runtime_end.update(time.time())
+
+  def process(self, element):
+    yield element
+
+
+class count_metrics(object):
+  def __init__(self, namespace, counter_name):
+    self.counter = Metrics.counter(namespace, counter_name)
+
+  def __call__(self, fn):
+    def decorated(*args):
+      element = args[1]
+      _, value = element
+      for i in range(len(value)):
+        self.counter.inc(i)
+      return fn(*args)
+
+    return decorated
diff --git a/sdks/python/apache_beam/testing/load_tests/pardo_test.py 
b/sdks/python/apache_beam/testing/load_tests/pardo_test.py
new file mode 100644
index 00000000000..d3c4e13c38e
--- /dev/null
+++ b/sdks/python/apache_beam/testing/load_tests/pardo_test.py
@@ -0,0 +1,158 @@
+#
+# 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 ParDo load test with Synthetic Source. Besides of the standard
+input options there are additional options:
+* number_of_counter_operations - number of pardo operations
+* metrics_project_id - the gcp project in case of saving metrics in big query
+* input_options - options for Synthetic Sources.
+
+Example test run on DirectRunner:
+
+python setup.py nosetests \
+    --test-pipeline-options="
+    --number_of_counter_operations=1000
+    --metrics_project_id=big-query-project
+    --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.pardo_test
+
+To run test on other runner (ex. Dataflow):
+
+python setup.py nosetests \
+    --test-pipeline-options="
+        --runner=TestDataflowRunner
+        --project=...
+        --staging_location=gs://...
+        --temp_location=gs://...
+        --sdk_location=./dist/apache-beam-x.x.x.dev0.tar.gz
+        --output=gc
+        --number_of_counter_operations=1000
+        --metrics_project_id=big-query-project
+        --input_options='{
+        \"num_records\": 1000,
+        \"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.pardo_test
+
+"""
+
+from __future__ import absolute_import
+
+import json
+import logging
+import unittest
+
+import apache_beam as beam
+from apache_beam.testing import synthetic_pipeline
+from apache_beam.testing.load_tests.load_test_metrics_utils import 
BigQueryMetricsCollector
+from apache_beam.testing.load_tests.load_test_metrics_utils import MeasureTime
+from apache_beam.testing.load_tests.load_test_metrics_utils import 
count_metrics
+from apache_beam.testing.test_pipeline import TestPipeline
+
+NAMESPACE = 'pardo'
+COUNTER_LABEL = "total_bytes_count"
+RUNTIME_LABEL = 'runtime'
+
+
+class ParDoTest(unittest.TestCase):
+
+  def parseTestPipelineOptions(self):
+    return {'numRecords': self.input_options.get('num_records'),
+            'keySizeBytes': self.input_options.get('key_size'),
+            'valueSizeBytes': self.input_options.get('value_size'),
+            'bundleSizeDistribution': {
+                'type': self.input_options.get(
+                    'bundle_size_distribution_type', 'const'
+                ),
+                'param': self.input_options.get(
+                    'bundle_size_distribution_param', 0
+                )
+            },
+            'forceNumInitialBundles': self.input_options.get(
+                'force_initial_num_bundles', 0
+            )
+           }
+
+  def setUp(self):
+    self.pipeline = TestPipeline(is_integration_test=True)
+    self.output = self.pipeline.get_option('output')
+    self.iterations = self.pipeline.get_option('number_of_counter_operations')
+    self.input_options = json.loads(self.pipeline.get_option('input_options'))
+
+    metrics_project_id = self.pipeline.get_option('metrics_project_id')
+    self.bigQuery = None
+    if metrics_project_id is not None:
+      schema = [{'name': RUNTIME_LABEL, 'type': 'FLOAT', 'mode': 'REQUIRED'},
+                {'name': COUNTER_LABEL, 'type': 'INTEGER', 'mode': 'REQUIRED'}]
+      self.bigQuery = BigQueryMetricsCollector(
+          metrics_project_id,
+          NAMESPACE,
+          schema
+      )
+
+  class _GetElement(beam.DoFn):
+    @count_metrics(namespace=NAMESPACE, counter_name=COUNTER_LABEL)
+    def process(self, element):
+      yield element
+
+  def testParDo(self):
+    if self.iterations is None:
+      num_runs = 1
+    else:
+      num_runs = int(self.iterations)
+
+    with self.pipeline as p:
+      pc = (p
+            | 'Read synthetic' >> beam.io.Read(
+                synthetic_pipeline.SyntheticSource(
+                    self.parseTestPipelineOptions()
+                ))
+            | 'Measure time' >> beam.ParDo(MeasureTime(NAMESPACE))
+           )
+
+      for i in range(num_runs):
+        label = 'Step: %d' % i
+        pc = (pc
+              | label >> beam.ParDo(self._GetElement()))
+
+      if self.output is not None:
+        # pylint: disable=expression-not-assigned
+        (pc
+         | "Write" >> beam.io.WriteToText(self.output)
+        )
+
+      result = p.run()
+      result.wait_until_finish()
+
+      if self.bigQuery is not None:
+        self.bigQuery.save_metrics(result)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  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: 162541)
    Time Spent: 20m  (was: 10m)

> Add integrations of Metrics API to Big Query for SyntheticSources load tests 
> in Python SDK
> ------------------------------------------------------------------------------------------
>
>                 Key: BEAM-5778
>                 URL: https://issues.apache.org/jira/browse/BEAM-5778
>             Project: Beam
>          Issue Type: Improvement
>          Components: testing
>            Reporter: Kasia Kucharczyk
>            Assignee: Kasia Kucharczyk
>            Priority: Major
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> Right now Metrics API collects basic metrics of load tests of 
> SyntheticSources (Python SDK). It should be collected in BigQuery for 
> presenting it on performance dashboards.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to