Skip to content

Commit

Permalink
[BEAM-6207] Added option to publish synthetic data to Kafka topic.
Browse files Browse the repository at this point in the history
SyntheticDataPubSubPublisher now accepts --kafkaBootstrapServerAddress option
which switches it to Kafka publishing mode instead of Google PubSub mode.
Key-Value pairs are published as strings.
  • Loading branch information
Michal Walenia committed Jan 24, 2019
1 parent b0cbd96 commit db59671
Show file tree
Hide file tree
Showing 2 changed files with 46 additions and 5 deletions.
2 changes: 2 additions & 0 deletions sdks/java/io/synthetic/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -29,11 +29,13 @@ dependencies {
shadow library.java.jackson_annotations
shadow library.java.jackson_databind
shadow library.java.guava
shadow library.java.kafka_clients

shadowTest library.java.vendored_guava_20_0
testCompile library.java.junit
testCompile library.java.hamcrest_core
testCompile library.java.hamcrest_library
shadow project(path: ":beam-sdks-java-core", configuration: "shadow")
shadow project(path: ":beam-runners-direct-java", configuration: "shadow")
shadow project(path: ":beam-sdks-java-io-kafka", configuration: "shadow")
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray;

import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import org.apache.beam.sdk.Pipeline;
Expand All @@ -30,6 +31,7 @@
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.kafka.KafkaIO;
import org.apache.beam.sdk.io.synthetic.SyntheticBoundedSource;
import org.apache.beam.sdk.io.synthetic.SyntheticOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticSourceOptions;
Expand All @@ -41,6 +43,7 @@
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.values.KV;
import org.apache.kafka.common.serialization.StringSerializer;

/**
* Pipeline that generates synthetic data and publishes it in PubSub topic.
Expand All @@ -50,9 +53,13 @@
* <pre>
* ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
* --insertionPipelineTopic=TOPIC_NAME
* --kafkaBootstrapServerAddress=SERVER_ADDRESS
* --sourceOptions={"numRecords":1000,...}'
* -PloadTest.mainClass="org.apache.beam.sdk.loadtests.SyntheticDataPubSubPublisher"
* </pre>
*
* Parameter kafkaBootstrapServerAddress is optional. If supplied, pipeline topic will be treated as
* Kafka topic name and records will be published to Kafka instead of PubSub.
*/
public class SyntheticDataPubSubPublisher {

Expand All @@ -73,6 +80,11 @@ public interface Options extends PipelineOptions, ApplicationNameOptions {
String getInsertionPipelineTopic();

void setInsertionPipelineTopic(String topic);

@Description("Kafka server address (optional)")
String getKafkaBootstrapServerAddress();

void setKafkaBootstrapServerAddress(String address);
}

public static void main(String[] args) throws IOException {
Expand All @@ -83,14 +95,41 @@ public static void main(String[] args) throws IOException {

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()));

if (options.getKafkaBootstrapServerAddress() != null) {
pipeline
.apply("Read synthetic data", Read.from(new SyntheticBoundedSource(sourceOptions)))
.apply("Map to Kafka messages", MapElements.via(new MapKVToString()))
.apply(
"Write to Kafka",
KafkaIO.<Void, String>write()
.withBootstrapServers(options.getKafkaBootstrapServerAddress())
.withTopic(options.getInsertionPipelineTopic())
.withValueSerializer(StringSerializer.class)
.values());
} else {
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();
}

public static class MapKVToString extends SimpleFunction<KV<byte[], byte[]>, String> {
@Override
public String apply(KV<byte[], byte[]> input) {
StringBuilder stringBuilder =
new StringBuilder()
.append("{")
.append(Arrays.toString(input.getKey()))
.append(",")
.append(Arrays.toString(input.getValue()))
.append("}");
return stringBuilder.toString();
}
}

private static class MapBytesToPubSubMessage
extends SimpleFunction<KV<byte[], byte[]>, PubsubMessage> {
@Override
Expand Down

0 comments on commit db59671

Please sign in to comment.