Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[BEAM-7503] Created CoGBK Python Load Test Jenkins job #8969

Merged
merged 3 commits into from
Jul 11, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
167 changes: 167 additions & 0 deletions .test-infra/jenkins/job_LoadTests_coGBK_Python.groovy
Original file line number Diff line number Diff line change
@@ -0,0 +1,167 @@
/*
* 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 CommonJobProperties as commonJobProperties
import CommonTestProperties
import LoadTestsBuilder as loadTestsBuilder
import PhraseTriggeringPostCommitBuilder
import CronJobBuilder

def now = new Date().format("MMddHHmmss", TimeZone.getTimeZone('UTC'))

def loadTestConfigurations = { datasetName -> [
[
title : 'CoGroupByKey Python Load test: 2GB of 100B records with a single key',
itClass : 'apache_beam.testing.load_tests.co_group_by_key_test:CoGroupByKeyTest.testCoGroupByKey',
runner : CommonTestProperties.Runner.DATAFLOW,
jobProperties: [
project : 'apache-beam-testing',
job_name : 'load-tests-python-dataflow-batch-cogbk-1-' + now,
temp_location : 'gs://temp-storage-for-perf-tests/loadtests',
publish_to_big_query : true,
metrics_dataset : datasetName,
metrics_table : "python_dataflow_batch_cogbk_1",
input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 1,' +
'"hot_key_fraction": 1}\'',
co_input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 5,' +
'"hot_key_fraction": 1}\'',
iterations : 1,
max_num_workers : 5,
num_workers : 5,
autoscaling_algorithm: 'NONE'
]
],
[
title : 'CoGroupByKey Python Load test: 2GB of 100B records with multiple keys',
itClass : 'apache_beam.testing.load_tests.co_group_by_key_test:CoGroupByKeyTest.testCoGroupByKey',
runner : CommonTestProperties.Runner.DATAFLOW,
jobProperties: [
project : 'apache-beam-testing',
job_name : 'load-tests-python-dataflow-batch-cogbk-2-' + now,
temp_location : 'gs://temp-storage-for-perf-tests/loadtests',
publish_to_big_query : true,
metrics_dataset : datasetName,
metrics_table : 'python_dataflow_batch_cogbk_2',
input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 5,' +
'"hot_key_fraction": 1}\'',
co_input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 5,' +
'"hot_key_fraction": 1}\'',
iterations : 1,
max_num_workers : 5,
num_workers : 5,
autoscaling_algorithm: 'NONE'
]
],
[
title : 'CoGroupByKey Python Load test: reiterate 4 times 10kB values',
itClass : 'apache_beam.testing.load_tests.co_group_by_key_test:CoGroupByKeyTest.testCoGroupByKey',
runner : CommonTestProperties.Runner.DATAFLOW,
jobProperties: [
project : 'apache-beam-testing',
job_name : 'load-tests-python-dataflow-batch-cogbk-3-' + now,
temp_location : 'gs://temp-storage-for-perf-tests/loadtests',
publish_to_big_query : true,
metrics_dataset : datasetName,
metrics_table : "python_dataflow_batch_cogbk_3",
input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 200000,' +
'"hot_key_fraction": 1}\'',
co_input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 5,' +
'"hot_key_fraction": 1}\'',
iterations : 4,
max_num_workers : 5,
num_workers : 5,
autoscaling_algorithm: 'NONE'
]
],
[
title : 'CoGroupByKey Python Load test: reiterate 4 times 2MB values',
itClass : 'apache_beam.testing.load_tests.co_group_by_key_test:CoGroupByKeyTest.testCoGroupByKey',
runner : CommonTestProperties.Runner.DATAFLOW,
jobProperties: [
project : 'apache-beam-testing',
job_name : 'load-tests-python-dataflow-batch-cogbk-4-' + now,
temp_location : 'gs://temp-storage-for-perf-tests/loadtests',
publish_to_big_query : true,
metrics_dataset : datasetName,
metrics_table : 'python_dataflow_batch_cogbk_4',
input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 1000,' +
'"hot_key_fraction": 1}\'',
co_input_options : '\'{' +
'"num_records": 20000000,' +
'"key_size": 10,' +
'"value_size": 90,' +
'"num_hot_keys": 5,' +
'"hot_key_fraction": 1}\'',
iterations : 4,
max_num_workers : 5,
num_workers : 5,
autoscaling_algorithm: 'NONE'
]
],
]}

def batchLoadTestJob = { scope, triggeringContext ->
scope.description('Runs Python CoGBK load tests on Dataflow runner in batch mode')
commonJobProperties.setTopLevelMainJobProperties(scope, 'master', 240)

def datasetName = loadTestsBuilder.getBigQueryDataset('load_test', triggeringContext)
for (testConfiguration in loadTestConfigurations(datasetName)) {
loadTestsBuilder.loadTest(scope, testConfiguration.title, testConfiguration.runner, CommonTestProperties.SDK.PYTHON, testConfiguration.jobProperties, testConfiguration.itClass)
}
}

CronJobBuilder.cronJob('beam_LoadTests_Python_CoGBK_Dataflow_Batch', 'H 16 * * *', this) {
batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.POST_COMMIT)
}

PhraseTriggeringPostCommitBuilder.postCommitJob(
'beam_LoadTests_Python_CoGBK_Dataflow_Batch',
'Run Load Tests Python CoGBK Dataflow Batch',
'Load Tests Python CoGBK Dataflow Batch suite',
this
) {
batchLoadTestJob(delegate, CommonTestProperties.TriggeringContext.PR)
}
29 changes: 20 additions & 9 deletions sdks/python/apache_beam/testing/load_tests/co_group_by_key_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,9 @@
* metrics_table (optional) - name of BigQuery table where metrics
will be stored,
* input_options - options for Synthetic Sources,
* co_input_options - options for Synthetic Sources.
* co_input_options - options for Synthetic Sources,
* iterations - number of reiterations over per-key-grouped values to perform
(default: 1).

Example test run on DirectRunner:

Expand All @@ -36,6 +38,7 @@
--publish_to_big_query=true
--metrics_dataset=python_load_tests
--metrics_table=co_gbk
--iterations=1
--input_options='{
\"num_records\": 1000,
\"key_size\": 5,
Expand All @@ -59,6 +62,7 @@
--project=...
--metrics_dataset=python_load_tests
--metrics_table=co_gbk
--iterations=1
--input_options=\'
{"num_records": 1,
"key_size": 1,
Expand Down Expand Up @@ -90,6 +94,7 @@
--publish_to_big_query=true
--metrics_dataset=python_load_tests
--metrics_table=co_gbk
--iterations=1
--input_options='{
\"num_records\": 1000,
\"key_size\": 5,
Expand All @@ -115,6 +120,7 @@
--project=...
--metrics_dataset=python_load_tests
--metrics_table=co_gbk
--iterations=1
--temp_location=gs://...
--input_options=\'
{"num_records": 1,
Expand Down Expand Up @@ -162,16 +168,20 @@ def setUp(self):
super(CoGroupByKeyTest, self).setUp()
self.co_input_options = json.loads(
self.pipeline.get_option('co_input_options'))
self.iterations = self.get_option_or_default('iterations', 1)

class _Ungroup(beam.DoFn):
def process(self, element):
class _UngroupAndReiterate(beam.DoFn):
def process(self, element, iterations):
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
for i in range(iterations):
for value in inputs:
if i == iterations - 1:
yield value
for value in co_inputs:
if i == iterations - 1:
yield value

def testCoGroupByKey(self):
pc1 = (self.pipeline
Expand All @@ -194,8 +204,9 @@ def testCoGroupByKey(self):
)
# pylint: disable=expression-not-assigned
({INPUT_TAG: pc1, CO_INPUT_TAG: pc2}
| 'CoGroupByKey: ' >> beam.CoGroupByKey()
| 'Consume Joined Collections' >> beam.ParDo(self._Ungroup())
| 'CoGroupByKey ' >> beam.CoGroupByKey()
| 'Consume Joined Collections' >> beam.ParDo(self._UngroupAndReiterate(),
self.iterations)
| 'Measure time: End' >> beam.ParDo(MeasureTime(self.metrics_namespace))
)

Expand Down
3 changes: 1 addition & 2 deletions sdks/python/scripts/run_pylint.sh
Original file line number Diff line number Diff line change
Expand Up @@ -74,8 +74,7 @@ pylint -j8 ${MODULE} --ignore-patterns="$FILES_TO_IGNORE"
echo "Running pycodestyle for module $MODULE:"
pycodestyle ${MODULE} --exclude="$FILES_TO_IGNORE"
echo "Running flake8 for module $MODULE:"
# TODO(BEAM-3959): Add F821 (undefined names) as soon as that test passes
flake8 ${MODULE} --count --select=E9,F822,F823 --show-source --statistics
flake8 ${MODULE} --count --select=E9,F821,F822,F823 --show-source --statistics

echo "Running isort for module $MODULE:"
# Skip files where isort is behaving weirdly
Expand Down