[
https://issues.apache.org/jira/browse/BEAM-5981?focusedWorklogId=163413&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-163413
]
ASF GitHub Bot logged work on BEAM-5981:
----------------------------------------
Author: ASF GitHub Bot
Created on: 07/Nov/18 11:05
Start Date: 07/Nov/18 11:05
Worklog Time Spent: 10m
Work Description: lgajowy closed pull request #6962: [BEAM-5981] Par do
load test
URL: https://github.com/apache/beam/pull/6962
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/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/CoGroupByKeyLoadTest.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/CoGroupByKeyLoadTest.java
index 01dfbadb0a3..85d27ea8123 100644
---
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/CoGroupByKeyLoadTest.java
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/CoGroupByKeyLoadTest.java
@@ -17,19 +17,12 @@
*/
package org.apache.beam.sdk.loadtests;
-import static org.apache.beam.sdk.loadtests.SyntheticUtils.applyStepIfPresent;
-import static org.apache.beam.sdk.loadtests.SyntheticUtils.createStep;
-import static org.apache.beam.sdk.loadtests.SyntheticUtils.fromJsonString;
-
import java.io.IOException;
import java.util.Optional;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
import
org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO.SyntheticSourceOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticStep;
import org.apache.beam.sdk.loadtests.metrics.MetricsMonitor;
-import org.apache.beam.sdk.loadtests.metrics.MetricsPublisher;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.options.Validation;
@@ -46,17 +39,17 @@
* Load test for {@link CoGroupByKey} operation.
*
* <p>The purpose of this test is to measure {@link CoGroupByKey}'s behaviour
in stressful
- * conditions. it uses {@link SyntheticBoundedIO} and {@link SyntheticStep}
which both can be
+ * conditions. It uses {@link SyntheticBoundedIO} and {@link SyntheticStep}
which both can be
* parametrized to generate keys and values of various size, impose delay
(sleep or cpu burnout) in
- * various moments during the pipeline execution and provide some other
performance challenges (see
- * Source's and Step's documentation for more details).
+ * various moments during the pipeline execution and provide some other
performance challenges.
*
* <p>In addition, this test allows to reiterate produced PCollection multiple
times to see how the
* pipeline behaves (e.g. if caches work etc.).
*
- * <p>To run it manually, use the following command:
- *
- * <pre>
+ * @see SyntheticStep
+ * @see SyntheticBoundedIO
+ * <p>To run it manually, use the following command:
+ * <pre>
* ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
* --iterations=1
* --sourceOptions={"numRecords":1000,...}
@@ -65,7 +58,7 @@
*
-PloadTest.mainClass="org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest"
* </pre>
*/
-public class CoGroupByKeyLoadTest {
+public class CoGroupByKeyLoadTest extends
LoadTest<CoGroupByKeyLoadTest.Options> {
private static final TupleTag<byte[]> INPUT_TAG = new TupleTag<>("input");
private static final TupleTag<byte[]> CO_INPUT_TAG = new
TupleTag<>("co-input");
@@ -87,19 +80,17 @@
void setIterations(Integer iterations);
}
- public static void main(String[] args) throws IOException {
- Options options = LoadTestOptions.readFromArgs(args, Options.class);
-
- SyntheticSourceOptions sourceOptions =
- fromJsonString(options.getSourceOptions(),
SyntheticSourceOptions.class);
+ private CoGroupByKeyLoadTest(String[] args) throws IOException {
+ super(args, Options.class, METRICS_NAMESPACE);
+ }
+ @Override
+ void loadTest() throws IOException {
SyntheticSourceOptions coSourceOptions =
fromJsonString(options.getCoSourceOptions(),
SyntheticSourceOptions.class);
Optional<SyntheticStep> syntheticStep =
createStep(options.getStepOptions());
- Pipeline pipeline = Pipeline.create(options);
-
PCollection<KV<byte[], byte[]>> input =
applyStepIfPresent(
pipeline.apply("Read input",
SyntheticBoundedIO.readFrom(sourceOptions)),
@@ -117,11 +108,6 @@ public static void main(String[] args) throws IOException {
.apply("CoGroupByKey", CoGroupByKey.create())
.apply("Ungroup and reiterate", ParDo.of(new
UngroupAndReiterate(options.getIterations())))
.apply("Collect metrics", ParDo.of(new
MetricsMonitor(METRICS_NAMESPACE)));
-
- PipelineResult result = pipeline.run();
- result.waitUntilFinish();
-
- MetricsPublisher.toConsole(result, METRICS_NAMESPACE);
}
private static class UngroupAndReiterate
@@ -157,4 +143,8 @@ public void processElement(ProcessContext c) {
}
}
}
+
+ public static void main(String[] args) throws IOException {
+ new CoGroupByKeyLoadTest(args).run();
+ }
}
diff --git
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/GroupByKeyLoadTest.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/GroupByKeyLoadTest.java
index 4cdfad055db..fb0bf841107 100644
---
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/GroupByKeyLoadTest.java
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/GroupByKeyLoadTest.java
@@ -18,18 +18,12 @@
package org.apache.beam.sdk.loadtests;
import static java.lang.String.format;
-import static org.apache.beam.sdk.loadtests.SyntheticUtils.createStep;
-import static org.apache.beam.sdk.loadtests.SyntheticUtils.fromJsonString;
import java.io.IOException;
import java.util.Optional;
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
-import
org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO.SyntheticSourceOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticStep;
import org.apache.beam.sdk.loadtests.metrics.MetricsMonitor;
-import org.apache.beam.sdk.loadtests.metrics.MetricsPublisher;
import org.apache.beam.sdk.options.Default;
import org.apache.beam.sdk.options.Description;
import org.apache.beam.sdk.transforms.DoFn;
@@ -42,26 +36,28 @@
* Load test for {@link GroupByKey} operation.
*
* <p>The purpose of this test is to measure {@link GroupByKey}'s behaviour in
stressful conditions.
- * it uses {@link SyntheticBoundedIO} and {@link SyntheticStep} which both can
be parametrized to
+ * It uses {@link SyntheticBoundedIO} and {@link SyntheticStep} which both can
be parametrized to
* generate keys and values of various size, impose delay (sleep or cpu
burnout) in various moments
- * during the pipeline execution and provide some other performance challenges
(see Source's and
- * Step's documentation for more details).
+ * during the pipeline execution and provide some other performance challenges.
*
- * <p>In addition, this test allows to: - fanout: produce one input (using
Synthetic Source) and
- * process it with multiple sessions performing the same set of operations -
reiterate produced
- * PCollection multiple times
- *
- * <p>To run it manually, use the following command:
- *
- * <pre>
+ * @see SyntheticStep
+ * @see SyntheticBoundedIO
+ * <p>In addition, this test allows to: - fanout: produce one input (using
Synthetic Source) and
+ * process it with multiple sessions performing the same set of operations
- reiterate produced
+ * PCollection multiple times
+ * <p>To run it manually, use the following command:
+ * <pre>
* ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
* --fanout=1
* --iterations=1
* --sourceOptions={"numRecords":1000,...}
* --stepOptions={"outputRecordsPerInputRecord":2...}'
+ * -PloadTest.mainClass="org.apache.beam.sdk.loadtests.GroupByKeyLoadTest"
* </pre>
*/
-public class GroupByKeyLoadTest {
+public class GroupByKeyLoadTest extends LoadTest<GroupByKeyLoadTest.Options> {
+
+ private static final String METRICS_NAMESPACE = "gbk";
/** Pipeline options for the test. */
public interface Options extends LoadTestOptions {
@@ -79,32 +75,25 @@
void setIterations(Integer iterations);
}
- public static void main(String[] args) throws IOException {
- Options options = LoadTestOptions.readFromArgs(args, Options.class);
-
- SyntheticSourceOptions sourceOptions =
- fromJsonString(options.getSourceOptions(),
SyntheticSourceOptions.class);
+ private GroupByKeyLoadTest(String[] args) throws IOException {
+ super(args, Options.class, METRICS_NAMESPACE);
+ }
+ @Override
+ void loadTest() throws IOException {
Optional<SyntheticStep> syntheticStep =
createStep(options.getStepOptions());
- Pipeline pipeline = Pipeline.create(options);
-
PCollection<KV<byte[], byte[]>> input =
pipeline.apply(SyntheticBoundedIO.readFrom(sourceOptions));
for (int branch = 0; branch < options.getFanout(); branch++) {
- SyntheticUtils.applyStepIfPresent(input, format("Synthetic step (%s)",
branch), syntheticStep)
- .apply(ParDo.of(new MetricsMonitor("gbk")))
+ applyStepIfPresent(input, format("Synthetic step (%s)", branch),
syntheticStep)
+ .apply(ParDo.of(new MetricsMonitor(METRICS_NAMESPACE)))
.apply(format("Group by key (%s)", branch), GroupByKey.create())
.apply(
format("Ungroup and reiterate (%s)", branch),
ParDo.of(new UngroupAndReiterate(options.getIterations())));
}
-
- PipelineResult result = pipeline.run();
- result.waitUntilFinish();
-
- MetricsPublisher.toConsole(result, "gbk");
}
private static class UngroupAndReiterate
@@ -131,4 +120,8 @@ public void processElement(ProcessContext c) {
}
}
}
+
+ public static void main(String[] args) throws IOException {
+ new GroupByKeyLoadTest(args).run();
+ }
}
diff --git
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticUtils.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
similarity index 52%
rename from
sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticUtils.java
rename to
sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
index 02089139496..b3f2d1f0288 100644
---
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticUtils.java
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
@@ -20,16 +20,68 @@
import com.fasterxml.jackson.databind.ObjectMapper;
import java.io.IOException;
import java.util.Optional;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
import org.apache.beam.sdk.io.synthetic.SyntheticOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticStep;
+import org.apache.beam.sdk.loadtests.metrics.MetricsPublisher;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
-/** Synthetic step utils related code. */
-class SyntheticUtils {
+/**
+ * Base class for all load tests. Provides common operations such as
initializing source/step
+ * options, creating a pipeline, etc.
+ */
+abstract class LoadTest<OptionsT extends LoadTestOptions> {
+
+ private String metricsNamespace;
+
+ OptionsT options;
+
+ SyntheticBoundedIO.SyntheticSourceOptions sourceOptions;
+
+ SyntheticStep.Options stepOptions;
+
+ Pipeline pipeline;
+
+ LoadTest(String[] args, Class<OptionsT> testOptions, String
metricsNamespace) throws IOException {
+ this.metricsNamespace = metricsNamespace;
+
+ this.options = LoadTestOptions.readFromArgs(args, testOptions);
+
+ this.sourceOptions =
+ fromJsonString(options.getSourceOptions(),
SyntheticBoundedIO.SyntheticSourceOptions.class);
+
+ this.stepOptions = fromJsonString(options.getStepOptions(),
SyntheticStep.Options.class);
+
+ this.pipeline = Pipeline.create(options);
+ }
+
+ /** The load test pipeline implementation. */
+ abstract void loadTest() throws IOException;
+
+ /** Runs the load test. */
+ public PipelineResult run() throws IOException {
+ loadTest();
+
+ PipelineResult result = pipeline.run();
+ result.waitUntilFinish();
+
+ MetricsPublisher.toConsole(result, metricsNamespace);
- static Optional<SyntheticStep> createStep(String stepOptions) throws
IOException {
+ return result;
+ }
+
+ <T extends SyntheticOptions> T fromJsonString(String json, Class<T> type)
throws IOException {
+ ObjectMapper mapper = new ObjectMapper();
+ T result = mapper.readValue(json, type);
+ result.validate();
+ return result;
+ }
+
+ Optional<SyntheticStep> createStep(String stepOptions) throws IOException {
if (stepOptions != null && !stepOptions.isEmpty()) {
return Optional.of(
new SyntheticStep(fromJsonString(stepOptions,
SyntheticStep.Options.class)));
@@ -38,7 +90,7 @@
}
}
- static PCollection<KV<byte[], byte[]>> applyStepIfPresent(
+ PCollection<KV<byte[], byte[]>> applyStepIfPresent(
PCollection<KV<byte[], byte[]>> input, String name,
Optional<SyntheticStep> syntheticStep) {
if (syntheticStep.isPresent()) {
@@ -47,12 +99,4 @@
return input;
}
}
-
- static <T extends SyntheticOptions> T fromJsonString(String json, Class<T>
type)
- throws IOException {
- ObjectMapper mapper = new ObjectMapper();
- T result = mapper.readValue(json, type);
- result.validate();
- return result;
- }
}
diff --git
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/ParDoLoadTest.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/ParDoLoadTest.java
new file mode 100644
index 00000000000..3916ad6eb11
--- /dev/null
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/ParDoLoadTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.
+ */
+package org.apache.beam.sdk.loadtests;
+
+import java.io.IOException;
+import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
+import org.apache.beam.sdk.io.synthetic.SyntheticStep;
+import org.apache.beam.sdk.loadtests.metrics.MetricsMonitor;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.Validation;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+
+/**
+ * Load test for {@link ParDo} operation.
+ *
+ * <p>The purpose of this test is to measure {@link ParDo}'s behaviour in
stressful conditions. It
+ * uses {@link SyntheticBoundedIO} and {@link SyntheticStep} which both can be
parametrized to
+ * generate keys and values of various size, impose delay (sleep or cpu
burnout) in various moments
+ * during the pipeline execution and provide some other performance challenges.
+ *
+ * @see SyntheticStep
+ * @see SyntheticBoundedIO
+ * <p>To run it manually, use the following command:
+ * <pre>
+ * ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
+ * --numberOfCounterOperations=1
+ * --sourceOptions={"numRecords":1000,...}
+ * --stepOptions={"outputRecordsPerInputRecord":2...}'
+ * -PloadTest.mainClass="org.apache.beam.sdk.loadtests.ParDoLoadTest"
+ * </pre>
+ */
+public class ParDoLoadTest extends LoadTest<ParDoLoadTest.Options> {
+
+ private static final String METRICS_NAMESPACE = "pardo";
+
+ /** Pipeline options specific for this test. */
+ interface Options extends LoadTestOptions {
+
+ @Description("Number consequent of ParDo operations (SyntheticSteps) to be
performed.")
+ @Default.Integer(1)
+ Integer getNumberOfCounterOperations();
+
+ void setNumberOfCounterOperations(Integer count);
+
+ @Override
+ @Description("Options for synthetic step")
+ @Validation.Required
+ String getStepOptions();
+ }
+
+ private ParDoLoadTest(String[] args) throws IOException {
+ super(args, Options.class, METRICS_NAMESPACE);
+ }
+
+ @Override
+ protected void loadTest() {
+ PCollection<KV<byte[], byte[]>> input =
+ pipeline.apply("Read input",
SyntheticBoundedIO.readFrom(sourceOptions));
+
+ for (int i = 0; i < options.getNumberOfCounterOperations(); i++) {
+ input = input.apply(String.format("Step: %d", i), ParDo.of(new
SyntheticStep(stepOptions)));
+ }
+
+ input.apply("Collect metrics", ParDo.of(new
MetricsMonitor(METRICS_NAMESPACE)));
+ }
+
+ public static void main(String[] args) throws IOException {
+ new ParDoLoadTest(args).run();
+ }
+}
----------------------------------------------------------------
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: 163413)
Time Spent: 0.5h (was: 20m)
> Create ParDo load test for Java SDK
> -----------------------------------
>
> Key: BEAM-5981
> URL: https://issues.apache.org/jira/browse/BEAM-5981
> Project: Beam
> Issue Type: Sub-task
> Components: testing
> Reporter: Lukasz Gajowy
> Assignee: Lukasz Gajowy
> Priority: Minor
> Time Spent: 0.5h
> Remaining Estimate: 0h
>
> This is more thoroughly described in this proposal:
> [https://docs.google.com/document/d/1PuIQv4v06eosKKwT76u7S6IP88AnXhTf870Rcj1AHt4/edit?usp=sharing]
>
> In short: this ticket is about implementing the ParDo load test that uses
> SyntheticStep and Synthetic source to create load on the pipeline.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)