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-6201] Data insertion pipeline #7238

Merged
merged 2 commits into from Dec 11, 2018
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
Expand Up @@ -355,4 +355,12 @@ public KV<byte[], byte[]> genKvPair(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;
}
}
1 change: 1 addition & 0 deletions sdks/java/testing/load-tests/build.gradle
Expand Up @@ -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")
Expand Down
Expand Up @@ -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;
Expand Down
Expand Up @@ -17,15 +17,15 @@
*/
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;
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.TimeMonitor;
import org.apache.beam.sdk.testutils.publishing.BigQueryResultsPublisher;
Expand Down Expand Up @@ -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(
Expand Down
@@ -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));
}
}
}