Skip to content
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
Original file line number Diff line number Diff line change
Expand Up @@ -104,9 +104,9 @@
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
import org.apache.kafka.common.TopicPartition;
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.checkerframework.checker.nullness.qual.Nullable;
import org.joda.time.Duration;
Expand Down Expand Up @@ -717,11 +717,12 @@ private void sinkEventsToKafka(PCollection<Event> events) {
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaTopic())
.withValueSerializer(ByteArraySerializer.class)
.withInputTimestamp()
.values());
}

static final DoFn<KV<Long, byte[]>, Event> BYTEARRAY_TO_EVENT =
new DoFn<KV<Long, byte[]>, Event>() {
static final DoFn<KV<byte[], byte[]>, Event> BYTEARRAY_TO_EVENT =
new DoFn<KV<byte[], byte[]>, Event>() {
@ProcessElement
public void processElement(ProcessContext c) throws IOException {
byte[] encodedEvent = c.element().getValue();
Expand All @@ -731,20 +732,35 @@ public void processElement(ProcessContext c) throws IOException {
};

/** Return source of events from Kafka. */
private PCollection<Event> sourceEventsFromKafka(Pipeline p, final Instant now) {
private PCollection<Event> sourceEventsFromKafka(Pipeline p, final Instant start) {
checkArgument((options.getBootstrapServers() != null), "Missing --bootstrapServers");
NexmarkUtils.console("Reading events from Kafka Topic %s", options.getKafkaTopic());

KafkaIO.Read<Long, byte[]> read =
KafkaIO.<Long, byte[]>read()
KafkaIO.Read<byte[], byte[]> read =
KafkaIO.<byte[], byte[]>read()
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaTopic())
.withKeyDeserializer(LongDeserializer.class)
.withKeyDeserializer(ByteArrayDeserializer.class)
.withValueDeserializer(ByteArrayDeserializer.class)
.withStartReadTime(now)
.withStartReadTime(start)
.withMaxNumRecords(
options.getNumEvents() != null ? options.getNumEvents() : Long.MAX_VALUE);

if (options.getKafkaTopicCreateTimeMaxDelaySec() >= 0) {
read =
read.withCreateTime(
Duration.standardSeconds(options.getKafkaTopicCreateTimeMaxDelaySec()));
}

if (options.getNumKafkaTopicPartitions() > 0) {
ArrayList<TopicPartition> partitionArrayList = new ArrayList<>();
for (int i = 0; i < options.getNumKafkaTopicPartitions(); ++i) {
partitionArrayList.add(new TopicPartition(options.getKafkaTopic(), i));
}
read = read.withTopicPartitions(partitionArrayList);
} else {
read = read.withTopic(options.getKafkaTopic());
}

return p.apply(queryName + ".ReadKafkaEvents", read.withoutMetadata())
.apply(queryName + ".KafkaToEvents", ParDo.of(BYTEARRAY_TO_EVENT));
}
Expand Down Expand Up @@ -802,6 +818,7 @@ private void sinkResultsToKafka(PCollection<String> formattedResults) {
.withBootstrapServers(options.getBootstrapServers())
.withTopic(options.getKafkaResultsTopic())
.withValueSerializer(StringSerializer.class)
.withInputTimestamp()
.values());
}

Expand Down Expand Up @@ -1012,7 +1029,8 @@ private PCollection<Event> createSource(Pipeline p, final Instant now) throws IO
// finished. In other case. when pubSubMode=SUBSCRIBE_ONLY, now should be null and
// it will be ignored.
source =
sourceEventsFromKafka(p, configuration.pubSubMode == COMBINED ? now : null);
sourceEventsFromKafka(
p, configuration.pubSubMode == COMBINED ? now : Instant.EPOCH);
} else {
source = sourceEventsFromPubsub(p);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -429,6 +429,20 @@ void setPubsubMessageSerializationMethod(

void setKafkaTopic(String value);

@Description(
"Number of partitions for Kafka topic in streaming mode. If unspecified, the broker will be queried for all partitions.")
int getNumKafkaTopicPartitions();

void setNumKafkaTopicPartitions(int value);

@Description(
"If non-negative, events from the Kafka topic will get their timestamps from the Kafka createtime, with the maximum delay for"
+ "disorder as specified.")
@Default.Integer(60)
int getKafkaTopicCreateTimeMaxDelaySec();

void setKafkaTopicCreateTimeMaxDelaySec(int value);

@Description("Base name of Kafka results topic in streaming mode.")
@Default.String("nexmark-results")
@Nullable
Expand Down