Skip to content

Commit

Permalink
[BEAM-6207] Refactored SyntheticDataPubSubPublisher into SyntheticDat…
Browse files Browse the repository at this point in the history
…aPublisher

- Publisher can now publish to both sinks simultaneously
- cleaned up code publishing data to sinks.
  • Loading branch information
Michal Walenia committed Jan 29, 2019
1 parent b2b0b28 commit ffaac57
Showing 1 changed file with 41 additions and 33 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,14 +36,14 @@
import org.apache.beam.sdk.io.synthetic.SyntheticOptions;
import org.apache.beam.sdk.io.synthetic.SyntheticSourceOptions;
import org.apache.beam.sdk.options.ApplicationNameOptions;
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.transforms.MapElements;
import org.apache.beam.sdk.transforms.SimpleFunction;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.kafka.common.serialization.StringSerializer;

/**
Expand All @@ -55,18 +55,21 @@
* ./gradlew :beam-sdks-java-load-tests:run -PloadTest.args='
* --insertionPipelineTopic=TOPIC_NAME
* --kafkaBootstrapServerAddress=SERVER_ADDRESS
* --kafkaTopic=KAFKA_TOPIC_NAME
* --sourceOptions={"numRecords":1000,...}'
* -PloadTest.mainClass="org.apache.beam.sdk.loadtests.SyntheticDataPubSubPublisher"
* -PloadTest.mainClass="org.apache.beam.sdk.loadtests.SyntheticDataPublisher"
* </pre>
*
* Parameter kafkaBootstrapServerAddress is optional. If provided, pipeline topic will be treated as
* Kafka topic name and records will be published to Kafka instead of PubSub.
* If parameters related to Kafka are provided, the publisher writes to Kafka. If both pubsub topic
* and Kafka params are present, records will be written to both sinks.
*/
public class SyntheticDataPubSubPublisher {
public class SyntheticDataPublisher {

private static final KvCoder<byte[], byte[]> RECORD_CODER =
KvCoder.of(ByteArrayCoder.of(), ByteArrayCoder.of());

private static Options options;

/** Options for the pipeline. */
public interface Options extends PipelineOptions, ApplicationNameOptions {

Expand All @@ -77,58 +80,63 @@ public interface Options extends PipelineOptions, ApplicationNameOptions {
void setSourceOptions(String sourceOptions);

@Description("PubSub topic to publish to")
@Validation.Required
String getInsertionPipelineTopic();

void setInsertionPipelineTopic(String topic);

@Description("Kafka server address")
@Default.String("")
String getKafkaBootstrapServerAddress();

void setKafkaBootstrapServerAddress(String address);

@Description("Kafka topic")
String getKafkaTopic();

void setKafkaTopic(String topic);
}

public static void main(String[] args) throws IOException {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);

SyntheticSourceOptions sourceOptions =
SyntheticOptions.fromJsonString(options.getSourceOptions(), SyntheticSourceOptions.class);

Pipeline pipeline = Pipeline.create(options);
PCollection<KV<byte[], byte[]>> syntheticData =
pipeline.apply("Read synthetic data", Read.from(new SyntheticBoundedSource(sourceOptions)));

if (!options.getKafkaBootstrapServerAddress().isEmpty()) {
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()));
if (options.getKafkaBootstrapServerAddress() != null && options.getKafkaTopic() != null) {
writeToKafka(syntheticData);
}
if (options.getInsertionPipelineTopic() != null) {
writeToPubSub(syntheticData);
}
pipeline.run().waitUntilFinish();
}

private static void writeToPubSub(PCollection<KV<byte[], byte[]>> collection) {
collection
.apply("Map to PubSub messages", MapElements.via(new MapBytesToPubSubMessage()))
.apply("Write to PubSub", PubsubIO.writeMessages().to(options.getInsertionPipelineTopic()));
}

private static void writeToKafka(PCollection<KV<byte[], byte[]>> collection) {
collection
.apply("Map to Kafka messages", MapElements.via(new MapKVToString()))
.apply(
"Write to Kafka",
KafkaIO.<Void, String>write()
.withBootstrapServers(options.getKafkaBootstrapServerAddress())
.withTopic(options.getKafkaTopic())
.withValueSerializer(StringSerializer.class)
.values());
}

private 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();
return String.format(
"{%s,%s}", Arrays.toString(input.getKey()), Arrays.toString(input.getValue()));
}
}

Expand Down

0 comments on commit ffaac57

Please sign in to comment.