[
https://issues.apache.org/jira/browse/BEAM-6201?focusedWorklogId=173986&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-173986
]
ASF GitHub Bot logged work on BEAM-6201:
----------------------------------------
Author: ASF GitHub Bot
Created on: 11/Dec/18 10:09
Start Date: 11/Dec/18 10:09
Worklog Time Spent: 10m
Work Description: lgajowy closed pull request #7238: [BEAM-6201] Data
insertion pipeline
URL: https://github.com/apache/beam/pull/7238
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/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
index e2f201aaac8d..fea1d1ce0824 100644
---
a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
+++
b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
@@ -355,4 +355,12 @@ public long nextDelay(long seed) {
random.nextBytes(val);
return KV.of(key, val);
}
+
+ public 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/build.gradle
b/sdks/java/testing/load-tests/build.gradle
index 198148c09683..c611747bdd65 100644
--- a/sdks/java/testing/load-tests/build.gradle
+++ b/sdks/java/testing/load-tests/build.gradle
@@ -51,6 +51,7 @@ dependencies {
shadow project(path: ":beam-runners-direct-java", configuration: "shadow")
shadow project(path: ":beam-sdks-java-io-synthetic", configuration: "shadow")
shadow project(path: ":beam-sdks-java-test-utils", configuration: "shadow")
+ compile project(path: ":beam-sdks-java-io-google-cloud-platform",
configuration: "shadow")
gradleRun project(path: project.path, configuration: "shadow")
gradleRun project(path: runnerDependency, configuration: "shadow")
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 0cb8f0e50ef5..f569b5aa6ed3 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,6 +17,8 @@
*/
package org.apache.beam.sdk.loadtests;
+import static org.apache.beam.sdk.io.synthetic.SyntheticOptions.fromJsonString;
+
import java.io.IOException;
import java.util.Optional;
import org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO;
diff --git
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
index 76ded71d4de9..9a948bf51eeb 100644
---
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
@@ -17,7 +17,8 @@
*/
package org.apache.beam.sdk.loadtests;
-import com.fasterxml.jackson.databind.ObjectMapper;
+import static org.apache.beam.sdk.io.synthetic.SyntheticOptions.fromJsonString;
+
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import java.io.IOException;
@@ -25,7 +26,6 @@
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.TimeMonitor;
import org.apache.beam.sdk.testutils.publishing.BigQueryResultsPublisher;
@@ -109,13 +109,6 @@ private static void checkBigQueryOptions(String dataset,
String table) {
table != null, "Please specify --bigQueryTable option if you want to
publish to BigQuery");
}
- <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(
diff --git
a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPubSubPublisher.java
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPubSubPublisher.java
new file mode 100644
index 000000000000..a42e8daed6fd
--- /dev/null
+++
b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPubSubPublisher.java
@@ -0,0 +1,107 @@
+/*
+ * 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.io.synthetic.SyntheticBoundedIO.SyntheticBoundedSource;
+import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;
+
+import java.io.IOException;
+import java.util.Collections;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.io.Read;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
+import
org.apache.beam.sdk.io.synthetic.SyntheticBoundedIO.SyntheticSourceOptions;
+import org.apache.beam.sdk.io.synthetic.SyntheticOptions;
+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;
+import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.values.KV;
+
+/**
+ * Pipeline that generates synthetic data and publishes it in PubSub topic.
+ *
+ * <p>To run it manually, use the following command:
+ *
+ * <pre>
+ * ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
+ * --insertionPipelineTopic=TOPIC_NAME
+ * --sourceOptions={"numRecords":1000,...}'
+ *
-PloadTest.mainClass="org.apache.beam.sdk.loadtests.SyntheticDataPubSubPublisher"
+ * </pre>
+ */
+public class SyntheticDataPubSubPublisher {
+
+ private static final KvCoder<byte[], byte[]> RECORD_CODER =
+ KvCoder.of(ByteArrayCoder.of(), ByteArrayCoder.of());
+
+ /** Options for the pipeline. */
+ public interface Options extends PipelineOptions, ApplicationNameOptions {
+
+ @Description("Options for synthetic source")
+ @Validation.Required
+ String getSourceOptions();
+
+ void setSourceOptions(String sourceOptions);
+
+ @Description("PubSub topic to publish to")
+ @Validation.Required
+ String getInsertionPipelineTopic();
+
+ void setInsertionPipelineTopic(String topic);
+ }
+
+ public static void main(String[] args) throws IOException {
+ Options options =
PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+ SyntheticSourceOptions sourceOptions =
+ SyntheticOptions.fromJsonString(options.getSourceOptions(),
SyntheticSourceOptions.class);
+
+ Pipeline pipeline = Pipeline.create(options);
+
+ pipeline
+ .apply("Read synthetic data", Read.from(new
SyntheticBoundedSource(sourceOptions)))
+ .apply("Map to PubSub messages", MapElements.via(new
MapBytesToPubSubMessage()))
+ .apply("Write to PubSub",
PubsubIO.writeMessages().to(options.getInsertionPipelineTopic()));
+
+ pipeline.run().waitUntilFinish();
+ }
+
+ private static class MapBytesToPubSubMessage
+ extends SimpleFunction<KV<byte[], byte[]>, PubsubMessage> {
+ @Override
+ public PubsubMessage apply(KV<byte[], byte[]> input) {
+ return new PubsubMessage(encodeInputElement(input),
Collections.emptyMap());
+ }
+ }
+
+ private static byte[] encodeInputElement(KV<byte[], byte[]> input) {
+ try {
+ return encodeToByteArray(RECORD_CODER, input);
+ } catch (CoderException e) {
+ throw new RuntimeException(String.format("Couldn't encode element.
Exception: %s", e));
+ }
+ }
+}
----------------------------------------------------------------
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: 173986)
Time Spent: 1h 10m (was: 1h)
> Provide a "Data insertion pipeline" for Python suites
> -----------------------------------------------------
>
> Key: BEAM-6201
> URL: https://issues.apache.org/jira/browse/BEAM-6201
> Project: Beam
> Issue Type: Sub-task
> Components: testing
> Reporter: Lukasz Gajowy
> Assignee: Lukasz Gajowy
> Priority: Trivial
> Fix For: 2.10.0
>
> Time Spent: 1h 10m
> Remaining Estimate: 0h
>
> We can provide a special pipeline that simply writes synthetic data to a
> PubSub topic. Then this data can be read from a subscription in a streaming
> test (eg. in python SDK tests or others).
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)