[ https://issues.apache.org/jira/browse/BEAM-5356?focusedWorklogId=162957&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-162957 ]
ASF GitHub Bot logged work on BEAM-5356: ---------------------------------------- Author: ASF GitHub Bot Created on: 06/Nov/18 09:04 Start Date: 06/Nov/18 09:04 Worklog Time Spent: 10m Work Description: lgajowy closed pull request #6942: [BEAM-5356] Co group by key load test URL: https://github.com/apache/beam/pull/6942 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 new file mode 100644 index 00000000000..01dfbadb0a3 --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/CoGroupByKeyLoadTest.java @@ -0,0 +1,160 @@ +/* + * 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 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; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TupleTag; + +/** + * 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 + * 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). + * + * <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> + * ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args=' + * --iterations=1 + * --sourceOptions={"numRecords":1000,...} + * --coSourceOptions={"numRecords":1000,...} + * --stepOptions={"outputRecordsPerInputRecord":2...}' + * -PloadTest.mainClass="org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest" + * </pre> + */ +public class CoGroupByKeyLoadTest { + + private static final TupleTag<byte[]> INPUT_TAG = new TupleTag<>("input"); + private static final TupleTag<byte[]> CO_INPUT_TAG = new TupleTag<>("co-input"); + private static final String METRICS_NAMESPACE = "co_gbk"; + + /** Pipeline options specific for this test. */ + public interface Options extends LoadTestOptions { + + @Description("Options for synthetic co-source.") + @Validation.Required + String getCoSourceOptions(); + + void setCoSourceOptions(String sourceOptions); + + @Description("Number of reiterations over per-key-grouped values to perform.") + @Default.Integer(1) + Integer getIterations(); + + 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); + + 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)), + "Synthetic step for input", + syntheticStep); + + PCollection<KV<byte[], byte[]>> coInput = + applyStepIfPresent( + pipeline.apply("Read co-input", SyntheticBoundedIO.readFrom(coSourceOptions)), + "Synthetic step for co-input", + syntheticStep); + + KeyedPCollectionTuple.of(INPUT_TAG, input) + .and(CO_INPUT_TAG, coInput) + .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 + extends DoFn<KV<byte[], CoGbkResult>, KV<byte[], byte[]>> { + + private int iterations; + + UngroupAndReiterate(int iterations) { + this.iterations = iterations; + } + + @ProcessElement + public void processElement(ProcessContext c) { + byte[] key = c.element().getKey(); + CoGbkResult elementValue = c.element().getValue(); + + Iterable<byte[]> inputs = elementValue.getAll(INPUT_TAG); + Iterable<byte[]> coInputs = elementValue.getAll(CO_INPUT_TAG); + + // Reiterate "iterations" times, emit output only once. + for (int i = 0; i < iterations; i++) { + for (byte[] value : inputs) { + if (i == iterations - 1) { + c.output(KV.of(key, value)); + } + } + + for (byte[] value : coInputs) { + if (i == iterations - 1) { + c.output(KV.of(key, value)); + } + } + } + } + } +} 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 48e595b0b59..4cdfad055db 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,28 +18,20 @@ package org.apache.beam.sdk.loadtests; import static java.lang.String.format; -import static org.apache.beam.sdk.loadtests.GroupByKeyLoadTest.Options.fromJsonString; -import static org.apache.beam.sdk.loadtests.GroupByKeyLoadTest.Options.readFromArgs; +import static org.apache.beam.sdk.loadtests.SyntheticUtils.createStep; +import static org.apache.beam.sdk.loadtests.SyntheticUtils.fromJsonString; -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.SyntheticBoundedIO.SyntheticSourceOptions; -import org.apache.beam.sdk.io.synthetic.SyntheticOptions; import org.apache.beam.sdk.io.synthetic.SyntheticStep; -import org.apache.beam.sdk.metrics.Counter; -import org.apache.beam.sdk.metrics.Distribution; -import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.ApplicationNameOptions; +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.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.testutils.metrics.MetricsReader; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; @@ -72,18 +64,7 @@ public class GroupByKeyLoadTest { /** Pipeline options for the test. */ - public interface Options extends PipelineOptions, ApplicationNameOptions { - - @Description("Options for synthetic source") - @Validation.Required - String getSourceOptions(); - - void setSourceOptions(String sourceOptions); - - @Description("Options for synthetic step") - String getStepOptions(); - - void setStepOptions(String stepOptions); + public interface Options extends LoadTestOptions { @Description("The number of GroupByKey operations to perform in parallel (fanout)") @Default.Integer(1) @@ -96,27 +77,15 @@ Integer getIterations(); void setIterations(Integer iterations); - - static Options readFromArgs(String[] args) { - return PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - } - - 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; - } } public static void main(String[] args) throws IOException { - Options options = readFromArgs(args); + Options options = LoadTestOptions.readFromArgs(args, Options.class); SyntheticSourceOptions sourceOptions = fromJsonString(options.getSourceOptions(), SyntheticSourceOptions.class); - Optional<SyntheticStep> syntheticStep = createSyntheticStep(options); + Optional<SyntheticStep> syntheticStep = createStep(options.getStepOptions()); Pipeline pipeline = Pipeline.create(options); @@ -124,8 +93,8 @@ public static void main(String[] args) throws IOException { pipeline.apply(SyntheticBoundedIO.readFrom(sourceOptions)); for (int branch = 0; branch < options.getFanout(); branch++) { - applySyntheticStep(input, branch, syntheticStep) - .apply(ParDo.of(new Monitor())) + SyntheticUtils.applyStepIfPresent(input, format("Synthetic step (%s)", branch), syntheticStep) + .apply(ParDo.of(new MetricsMonitor("gbk"))) .apply(format("Group by key (%s)", branch), GroupByKey.create()) .apply( format("Ungroup and reiterate (%s)", branch), @@ -135,37 +104,7 @@ public static void main(String[] args) throws IOException { PipelineResult result = pipeline.run(); result.waitUntilFinish(); - printMetrics(result); - } - - private static void printMetrics(PipelineResult result) { - MetricsReader resultMetrics = new MetricsReader(result, "gbk"); - - long totalBytes = resultMetrics.getCounterMetric("totalBytes.count", -1); - long startTime = resultMetrics.getStartTimeMetric(System.currentTimeMillis(), "runtime"); - long endTime = resultMetrics.getEndTimeMetric(System.currentTimeMillis(), "runtime"); - - System.out.println(String.format("Total bytes: %s", totalBytes)); - System.out.println(String.format("Total time (millis): %s", endTime - startTime)); - } - - private static PCollection<KV<byte[], byte[]>> applySyntheticStep( - PCollection<KV<byte[], byte[]>> input, int branch, Optional<SyntheticStep> syntheticStep) { - - if (syntheticStep.isPresent()) { - return input.apply(format("Synthetic step (%s)", branch), ParDo.of(syntheticStep.get())); - } else { - return input; - } - } - - private static Optional<SyntheticStep> createSyntheticStep(Options options) throws IOException { - if (options.getStepOptions() != null && !options.getStepOptions().isEmpty()) { - return Optional.of( - new SyntheticStep(fromJsonString(options.getStepOptions(), SyntheticStep.Options.class))); - } else { - return Optional.empty(); - } + MetricsPublisher.toConsole(result, "gbk"); } private static class UngroupAndReiterate @@ -192,19 +131,4 @@ public void processElement(ProcessContext c) { } } } - - private static class Monitor extends DoFn<KV<byte[], byte[]>, KV<byte[], byte[]>> { - - private static final String NAMESPACE = "gbk"; - - private final Distribution timeDistribution = Metrics.distribution(NAMESPACE, "runtime"); - private final Counter totalBytes = Metrics.counter(NAMESPACE, "totalBytes.count"); - - @ProcessElement - public void processElement(ProcessContext c) { - timeDistribution.update(System.currentTimeMillis()); - totalBytes.inc(c.element().getKey().length + c.element().getValue().length); - c.output(c.element()); - } - } } diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTestOptions.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTestOptions.java new file mode 100644 index 00000000000..5654666b82a --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTestOptions.java @@ -0,0 +1,43 @@ +/* + * 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 org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; + +/** Common interface for all load test options. */ +public interface LoadTestOptions extends PipelineOptions, ApplicationNameOptions { + + @Description("Options for synthetic source") + @Validation.Required + String getSourceOptions(); + + void setSourceOptions(String sourceOptions); + + @Description("Options for synthetic step") + String getStepOptions(); + + void setStepOptions(String stepOptions); + + static <T extends LoadTestOptions> T readFromArgs(String[] args, Class<T> optionsClass) { + return PipelineOptionsFactory.fromArgs(args).withValidation().as(optionsClass); + } +} 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/SyntheticUtils.java new file mode 100644 index 00000000000..02089139496 --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticUtils.java @@ -0,0 +1,58 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.util.Optional; +import org.apache.beam.sdk.io.synthetic.SyntheticOptions; +import org.apache.beam.sdk.io.synthetic.SyntheticStep; +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 { + + static Optional<SyntheticStep> createStep(String stepOptions) throws IOException { + if (stepOptions != null && !stepOptions.isEmpty()) { + return Optional.of( + new SyntheticStep(fromJsonString(stepOptions, SyntheticStep.Options.class))); + } else { + return Optional.empty(); + } + } + + static PCollection<KV<byte[], byte[]>> applyStepIfPresent( + PCollection<KV<byte[], byte[]>> input, String name, Optional<SyntheticStep> syntheticStep) { + + if (syntheticStep.isPresent()) { + return input.apply(name, ParDo.of(syntheticStep.get())); + } else { + 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/metrics/MetricsMonitor.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/MetricsMonitor.java new file mode 100644 index 00000000000..b1785f2f8fe --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/MetricsMonitor.java @@ -0,0 +1,44 @@ +/* + * 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.metrics; + +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; + +/** Monitors various metrics from within a pipeline. */ +public class MetricsMonitor extends DoFn<KV<byte[], byte[]>, KV<byte[], byte[]>> { + + private Distribution timeDistribution; + + private Counter totalBytes; + + public MetricsMonitor(String namespace) { + this.timeDistribution = Metrics.distribution(namespace, "runtime"); + this.totalBytes = Metrics.counter(namespace, "totalBytes.count"); + } + + @ProcessElement + public void processElement(ProcessContext c) { + timeDistribution.update(System.currentTimeMillis()); + totalBytes.inc(c.element().getKey().length + c.element().getValue().length); + c.output(c.element()); + } +} diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/MetricsPublisher.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/MetricsPublisher.java new file mode 100644 index 00000000000..a6c05abbd01 --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/MetricsPublisher.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.testutils.metrics.MetricsReader; + +/** Provides ways to publish metrics gathered during test invocation. */ +public class MetricsPublisher { + + public static void toConsole(PipelineResult result, String namespace) { + MetricsReader resultMetrics = new MetricsReader(result, namespace); + + long totalBytes = resultMetrics.getCounterMetric("totalBytes.count", -1); + long startTime = resultMetrics.getStartTimeMetric(System.currentTimeMillis(), "runtime"); + long endTime = resultMetrics.getEndTimeMetric(System.currentTimeMillis(), "runtime"); + + System.out.println(String.format("Total bytes: %s", totalBytes)); + System.out.println(String.format("Total time (millis): %s", endTime - startTime)); + } +} diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/package-info.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/package-info.java new file mode 100644 index 00000000000..309e820c602 --- /dev/null +++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/metrics/package-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ +/** Metrics related API for load tests. */ +package org.apache.beam.sdk.loadtests.metrics; ---------------------------------------------------------------- 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: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 162957) Time Spent: 0.5h (was: 20m) > Create CoGroupByKey load test for Java SDK > ------------------------------------------ > > Key: BEAM-5356 > URL: https://issues.apache.org/jira/browse/BEAM-5356 > Project: Beam > Issue Type: New Feature > 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 CoGroupByKeyLoadIT that uses > SyntheticStep and Synthetic source to create load on the pipeline. -- This message was sent by Atlassian JIRA (v7.6.3#76005)