Skip to content

Commit

Permalink
Merge pull request #7238: [BEAM-6201] Data insertion pipeline
Browse files Browse the repository at this point in the history
  • Loading branch information
lgajowy committed Dec 11, 2018
2 parents b592f94 + cd8f014 commit b0069eb
Show file tree
Hide file tree
Showing 5 changed files with 120 additions and 9 deletions.
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));
}
}
}

0 comments on commit b0069eb

Please sign in to comment.