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-2852] Add support for Kafka as source/sink on Nexmark #5019

Merged
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.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 2 additions & 0 deletions sdks/java/nexmark/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ dependencies {
shadow project(path: ":beam-sdks-java-io-google-cloud-platform", configuration: "shadow")
shadow project(path: ":beam-sdks-java-extensions-google-cloud-platform-core", configuration: "shadow")
shadow project(path: ":beam-sdks-java-extensions-sql", configuration: "shadow")
shadow project(path: ":beam-sdks-java-io-kafka", configuration: "shadow")
shadow library.java.google_api_services_bigquery
shadow library.java.jackson_core
shadow library.java.jackson_annotations
Expand All @@ -38,6 +39,7 @@ dependencies {
shadow library.java.junit
shadow library.java.hamcrest_core
shadow library.java.commons_lang3
shadow library.java.kafka_clients
shadow project(path: ":beam-runners-direct-java", configuration: "shadow")
shadow library.java.slf4j_jdk14
testCompile library.java.hamcrest_core
Expand Down
31 changes: 29 additions & 2 deletions sdks/java/nexmark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -192,6 +192,15 @@
<argLine>-da</argLine> <!-- disable assert in Calcite converter validation -->
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<configuration>
<source>1.8</source>
<target>1.8</target>
</configuration>
</plugin>
</plugins>
</build>

Expand Down Expand Up @@ -274,13 +283,13 @@
<artifactId>hamcrest-core</artifactId>
<scope>compile</scope>
</dependency>

<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-library</artifactId>
<scope>compile</scope>
</dependency>

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-direct-java</artifactId>
Expand All @@ -303,5 +312,23 @@
<artifactId>commons-lang3</artifactId>
<scope>runtime</scope>
</dependency>

<dependency>
<groupId>com.google.auto.value</groupId>
<artifactId>auto-value</artifactId>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>beam-sdks-java-io-kafka</artifactId>
</dependency>

<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>${kafka.clients.version}</version>
</dependency>

</dependencies>
</project>
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
*/
package org.apache.beam.sdk.nexmark;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;

import com.google.api.services.bigquery.model.TableFieldSchema;
Expand All @@ -42,6 +43,7 @@
import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO;
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.metrics.DistributionResult;
import org.apache.beam.sdk.metrics.MetricNameFilter;
import org.apache.beam.sdk.metrics.MetricQueryResults;
Expand Down Expand Up @@ -88,11 +90,16 @@
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionTuple;
import org.apache.beam.sdk.values.TimestampedValue;
import org.apache.beam.sdk.values.TupleTag;
import org.apache.beam.sdk.values.TupleTagList;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.LongDeserializer;
import org.apache.kafka.common.serialization.StringSerializer;
import org.joda.time.Duration;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -764,6 +771,69 @@ public void processElement(ProcessContext c) {
}));
}

static final DoFn<Event, byte[]> EVENT_TO_BYTEARRAY =
new DoFn<Event, byte[]>() {
@ProcessElement
public void processElement(ProcessContext c) {
try {
byte[] encodedEvent = CoderUtils.encodeToByteArray(Event.CODER, c.element());
c.output(encodedEvent);
} catch (CoderException e1) {
LOG.error("Error while sending Event {} to Kafka: serialization error",
c.element().toString());
}
}
};

/**
* Send {@code events} to Kafka.
*/
private void sinkEventsToKafka(PCollection<Event> events) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you please wire this method up and add a COMBINED mode similar to what is done in Pub/Sub?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMHO I think we should refactor the whole COMBINED mode:

  • See NexmarkLauncher#createSource: it does a switch on the source type to configure sink.
  • it sends synthetic events to sink when in COMBINED mode but NexmarkUtils#COMBINED states that combine modes is for "Both publish and consume, but as separate jobs".
    Once refactored to something more coherent, implement it for kafka.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what is important in the connection with MOM:

  • have the ability to keep a track of the generated events that lead to a benchmark result
  • be able to read events from a topic
  • write benchmark results to topic

PCollection<byte[]> eventToBytes =
events.apply("Event to bytes", ParDo.of(EVENT_TO_BYTEARRAY));
eventToBytes.apply(KafkaIO.<Void, byte[]>write()
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaSinkTopic())
.withValueSerializer(ByteArraySerializer.class)
.values());

}


static final DoFn<KV<Long, byte[]>, Event> BYTEARRAY_TO_EVENT =
new DoFn<KV<Long, byte[]>, Event>() {
@ProcessElement
public void processElement(ProcessContext c) {
byte[] encodedEvent = c.element().getValue();
try {
Event event = CoderUtils.decodeFromByteArray(Event.CODER, encodedEvent);
c.output(event);
} catch (CoderException e) {
LOG.error("Error while decoding Event from Kafka message: serialization error");
}
}
};

/**
* Return source of events from Kafka.
*/
private PCollection<Event> sourceEventsFromKafka(Pipeline p) {
NexmarkUtils.console("Reading events from Kafka Topic %s", options.getKafkaSourceTopic());

checkArgument(!Strings.isNullOrEmpty(options.getBootstrapServers()),
"Missing --bootstrapServers");

KafkaIO.Read<Long, byte[]> read = KafkaIO.<Long, byte[]>read()
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaSourceTopic())
.withKeyDeserializer(LongDeserializer.class)
.withValueDeserializer(ByteArrayDeserializer.class);

return p
.apply(queryName + ".ReadKafkaEvents", read.withoutMetadata())
.apply(queryName + ".KafkaToEvents", ParDo.of(BYTEARRAY_TO_EVENT));
}

/**
* Return Avro source of events from {@code options.getInputFilePrefix}.
*/
Expand Down Expand Up @@ -813,6 +883,22 @@ public void processElement(ProcessContext c) {
.apply(queryName + ".WritePubsubEvents", io);
}

/**
* Send {@code formattedResults} to Kafka.
*/
private void sinkResultsToKafka(PCollection<String> formattedResults) {
checkArgument(!Strings.isNullOrEmpty(options.getBootstrapServers()),
"Missing --bootstrapServers");

formattedResults.apply(
queryName + ".WriteKafkaResults",
KafkaIO.<Void, String>write()
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaSinkTopic())
.withValueSerializer(StringSerializer.class)
.values());
}

/**
* Send {@code formattedResults} to Pubsub.
*/
Expand Down Expand Up @@ -923,6 +1009,9 @@ private PCollection<Event> createSource(Pipeline p, final long now) {
case AVRO:
source = sourceEventsFromAvro(p);
break;
case KAFKA:
source = sourceEventsFromKafka(p);
break;
case PUBSUB:
// Setup the sink for the publisher.
switch (configuration.pubSubMode) {
Expand Down Expand Up @@ -1010,6 +1099,9 @@ private void sink(PCollection<TimestampedValue<KnownSize>> results, long now) {
case PUBSUB:
sinkResultsToPubsub(formattedResults, now);
break;
case KAFKA:
sinkResultsToKafka(formattedResults);
break;
case TEXT:
sinkResultsToText(formattedResults, now);
break;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -406,4 +406,24 @@ public interface NexmarkOptions
String getQueryLanguage();

void setQueryLanguage(String value);

@Description("Base name of Kafka source topic in streaming mode.")
@Nullable
@Default.String("nexmark-source")
String getKafkaSourceTopic();

void setKafkaSourceTopic(String value);

@Description("Base name of Kafka sink topic in streaming mode.")
@Nullable
@Default.String("nexmark-sink")
String getKafkaSinkTopic();

void setKafkaSinkTopic(String value);

@Description("Kafka Bootstrap Server domains.")
@Nullable
String getBootstrapServers();

void setBootstrapServers(String value);
}
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,11 @@ public enum SourceType {
/**
* Read from a PubSub topic. It will be fed the same synthetic events by this pipeline.
*/
PUBSUB
PUBSUB,
/**
* Read events from a Kafka topic. It will be fed the same synthetic events by this pipeline.
*/
KAFKA
}

/**
Expand All @@ -118,6 +122,10 @@ public enum SinkType {
* Write to a PubSub topic. It will be drained by this pipeline.
*/
PUBSUB,
/**
* Write to a Kafka topic. It will be drained by this pipeline.
*/
KAFKA,
/**
* Write to a text file. Only works in batch mode.
*/
Expand All @@ -129,7 +137,7 @@ public enum SinkType {
/**
* Write raw Events to BigQuery.
*/
BIGQUERY,
BIGQUERY
}

/**
Expand Down