From f1e2580ac8391594437928cc8e7eec394c54968c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 16 Feb 2016 14:37:56 -0800 Subject: [PATCH 01/72] pom.xml --- contrib/kafka/pom.xml | 158 ++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 158 insertions(+) create mode 100644 contrib/kafka/pom.xml diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml new file mode 100644 index 0000000000..94016c1f91 --- /dev/null +++ b/contrib/kafka/pom.xml @@ -0,0 +1,158 @@ + + + + 4.0.0 + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka + Google Cloud Dataflow Kafka Connectors + Library to read Kafka topics from Dataflow + 0.0.1-SNAPSHOT + jar + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + [1.2.0,2.0.0) + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + + com.puppycrawl.tools + checkstyle + 6.6 + + + + ../../checkstyle.xml + true + true + true + + + + + check + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + Google Cloud Dataflow Kafka Contrib + Google Cloud Dataflow Kafka Contrib + + com.google.cloud.dataflow.contrib.kafka + false + ]]> + + + + https://cloud.google.com/dataflow/java-sdk/JavaDoc/ + ${basedir}/../../javadoc/dataflow-sdk-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ + ${basedir}/../../javadoc/guava-docs + + + + + + + jar + + package + + + + + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${google-cloud-dataflow-version} + + + + org.apache.kafka + kafka-clients + [0.9,) + + + + + junit + junit + 4.11 + test + + + From 74548166b7036e5e2f068d80c98e5969e40fb58f Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 18 Feb 2016 13:29:42 -0800 Subject: [PATCH 02/72] most of the implemetation --- .../contrib/kafka/KafkaCheckpointMark.java | 75 +++ .../dataflow/contrib/kafka/KafkaSource.java | 490 ++++++++++++++++++ 2 files changed, 565 insertions(+) create mode 100644 contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java create mode 100644 contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java new file mode 100644 index 0000000000..eff1b03b9c --- /dev/null +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java @@ -0,0 +1,75 @@ +package com.google.cloud.dataflow.contrib.kafka; + +import java.io.IOException; +import java.util.List; + +import org.apache.kafka.common.TopicPartition; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; + +/** + * Checkpoint for an unbounded KafkaSource reader. Consists of Kafka topic name, partition id, + * and the latest offset consumed so far. + * + * @author rangadi + */ +@DefaultCoder(AvroCoder.class) +public class KafkaCheckpointMark implements UnboundedSource.CheckpointMark { + + private final List partitions; + + public KafkaCheckpointMark(List partitions) { + this.partitions = partitions; + } + + @SuppressWarnings("unused") // for AvroCoder + private KafkaCheckpointMark() { + partitions = null; + } + + public List getPartitions() { + return partitions; + } + + @Override + public void finalizeCheckpoint() throws IOException { + /* + * nothing to do. + * we might want to support committing offset in Kafka, though it does not guarantee + * no-duplicates, it could support Dataflow restart better. + * Unlike an update of a dataflow job, a restart does not have checkpoint state. + * This secondary checkpoint might be a good start for readers. + * Another similar benefit is when the number of workers or number of Kafka partitions + * changes. + */ + } + + public static class PartitionMark { + private final TopicPartition topicPartition; + private final long offset; + + /** + * TODO + * @param topic + * @param partition + * @param offset + * @param consumed + */ + public PartitionMark(TopicPartition topicPartition, long offset) { + this.topicPartition = topicPartition; + this.offset = offset; + } + + public TopicPartition getTopicPartition() { + return topicPartition; + } + + public long getOffset() { + return offset; + } + } + +} + diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java new file mode 100644 index 0000000000..552863fb25 --- /dev/null +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -0,0 +1,490 @@ +/* + * Copyright (C) 2015 The Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.contrib.kafka; + +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import javax.annotation.Nullable; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.cloud.dataflow.sdk.coders.AvroCoder; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; +import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.io.Closeables; + +/** + * TODO(rangadi) + * + * @author rangadi + */ +public class KafkaSource { + + private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); + + /* TODO: + * - abstract out kafka interactions + * - should we let user specify key and value deserializers in kafka. I don't think so. key + * matters to Kafka only at the producer (hashCode is used for partition). the consumer does + * not care. We could use DataFlow coder or SerializableFunction. + * - leave a comment about optionally storing + */ + + private static class IdentityFn implements SerializableFunction { + public T apply(T input) { + return input; + } + } + + /** + * A function that returns {@link Instant#now} as the timestamp for each generated element. + */ + private static class NowTimestampFn implements SerializableFunction { + @Override + public Instant apply(T input) { + return Instant.now(); + } + } + + public static Builder unboundedSourceBuilder() { + return new Builder(); + } + + public static Builder unboundedByteSourceBuilder() { + return new Builder() + .withKeyDecoderFn(new IdentityFn()) + .withValueDecoderFn(new IdentityFn()); + } + + public static class Builder { + + private List topics; + // future: let users specify subset of partitions to read + private SerializableFunction keyDecoderFn; + private SerializableFunction valueDecoderFn; + private SerializableFunction, Instant> timestampFn = + new NowTimestampFn>(); // default processing timestamp + + private ImmutableMap.Builder consumerConfigBuilder = ImmutableMap + .builder() + .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()) + .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()) + .put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") // default to latest offset when last offset is unknown. + .put("enable.auto.commit", false); // disable auto commit (may be enabled by the user) + + /** + * set of properties that are not required or don't make sense + */ + private static final Map ignoredConsumerProperties = ImmutableMap.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDecoderFn instead" + + // "group.id", "enable.auto.commit", "auto.commit.interval.ms" : + // lets allow these, applications can have better resume point for restarts. + ); + + + private Builder() {} + + /** + * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). + */ + public Builder withBootstrapServers(String bootstrapServers) { + return withConsumerConfig("bootstrap.servers", bootstrapServers); + } + + /** + * Set Kafka topics to be consumed. This is required. + */ + public Builder withTopics(Collection topics) { + this.topics = ImmutableList.copyOf(topics); + return this; + } + + /** + * Set a {@KafkaConsumer} configuration properties. + * @see ConsumerConfig + */ + public Builder withConsumerConfig(String configKey, Object configValue) { + Preconditions.checkArgument(ignoredConsumerProperties.containsKey(configKey), + "No need to configure '%s'. %s", configKey, ignoredConsumerProperties.get(configKey)); + consumerConfigBuilder.put(configKey, configValue); + return this; + } + + public Builder withKeyDecoderFn( + SerializableFunction keyDecoderFn) { + this.keyDecoderFn = keyDecoderFn; + return this; + } + + public Builder withValueDecoderFn( + SerializableFunction valueDecoderFn) { + this.valueDecoderFn = valueDecoderFn; + return this; + } + + /** + * Set a timestamp function. Default is the timestamp when the ConsumerRecord is processed + * by {@UnboundedReader#advance()} + */ + public Builder withTimestampFn( + SerializableFunction, Instant> timestampFn) { + this.timestampFn = timestampFn; + return this; + } + + public UnboundedSource, KafkaCheckpointMark> build() { + + ImmutableMap consumerConfig = consumerConfigBuilder.build(); + + Preconditions.checkNotNull( + consumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), + "Kafka bootstrap servers should be set"); + Preconditions.checkNotNull(topics, "Kafka topics should be set"); + Preconditions.checkArgument(!topics.isEmpty(), "At least one topic is required"); + Preconditions.checkNotNull(keyDecoderFn, "Decoder for Kafka key bytes should be set"); + Preconditions.checkNotNull(valueDecoderFn, "Decoder for Kafka values bytes should be set"); + + return new UnboundedKafkaSource( + consumerConfig, + topics, + keyDecoderFn, + valueDecoderFn, + timestampFn, + ImmutableList.of() // no assigned partitions yet + ); + } + } + + /** Static class, prevent instantiation */ + private KafkaSource() {} + + private static class UnboundedKafkaSource + extends UnboundedSource, KafkaCheckpointMark> { + + private final ImmutableMap consumerConfig; + private final List topics; + private final SerializableFunction keyDecoderFn; + private final SerializableFunction valueDecoderFn; + private final SerializableFunction, Instant> timestampFn; + private final List assignedPartitions; + + public UnboundedKafkaSource( + ImmutableMap consumerConfig, + List topics, + SerializableFunction keyDecoderFn, + SerializableFunction valueDecoderFn, + SerializableFunction, Instant> timestampFn, + List assignedPartitions) { + + this.consumerConfig = consumerConfig; + this.topics = topics; + this.keyDecoderFn = keyDecoderFn; + this.valueDecoderFn = valueDecoderFn; + this.timestampFn = timestampFn; + this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); + } + + @Override + public List, KafkaCheckpointMark>> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + + // XXX : I was checking various Java 8 streams and collectors.. thats is the reason for heavy use them here :) + + KafkaConsumer consumer = new KafkaConsumer(consumerConfig); + + List partitions; + + try { + // fetch partitions for each topic and sort them in order. + // sort by partitionId so that topics are evenly distributed among the splits. + + partitions = topics + .stream() + .flatMap(topic -> consumer.partitionsFor(topic).stream()) + .map(partInfo -> new TopicPartition(partInfo.topic(), partInfo.partition())) + .sorted((p1, p2) -> ComparisonChain.start() // sort by + .compare(p1.partition(), p2.partition()) + .compare(p1.topic(), p2.topic()) + .result()) + .collect(Collectors.toList()); + } finally { + consumer.close(); + } + + Preconditions.checkArgument(desiredNumSplits > 0); + Preconditions.checkState(partitions.size() > 0, + "Could not find any partitions. Please check Kafka configuration and topic names"); + + int numSplits = Math.min(desiredNumSplits, partitions.size()); + + Map> assignments = IntStream.range(0, partitions.size()) + .mapToObj(i -> i) + .collect(Collectors.groupingBy(i -> i % numSplits)); // groupingBy preserves order. + + // create a new source for each split with the assigned partitions for the split + return IntStream.range(0, numSplits) + .mapToObj(split -> { + + List assignedToSplit = assignments.get(split) + .stream() + .map(i -> partitions.get(i)) + .collect(Collectors.toList()); + + LOG.info("Partitions assigned for split %d : %s", + split, Joiner.on(",").join(assignedToSplit)); + + // copy of 'this', except for assignedPartitions, which is replaced by assignedToSplit + return new UnboundedKafkaSource( + this.consumerConfig, + this.topics, + this.keyDecoderFn, + this.valueDecoderFn, + this.timestampFn, + assignedToSplit); + }) + .collect(Collectors.toList()); + } + + @Override + public UnboundedReader> createReader( + PipelineOptions options, + KafkaCheckpointMark checkpointMark) { + return new UnboundedKafkaReader(this, checkpointMark); + } + + @Override + public Coder getCheckpointMarkCoder() { + return AvroCoder.of(KafkaCheckpointMark.class); + } + + @Override + public boolean requiresDeduping() { + return false; + } + + @Override + public void validate() { + // TODO anything to do here? + } + + @Override + public Coder> getDefaultOutputCoder() { + // no coder required. user explicitly provides functions to decode key and value + return null; + } + } + + private static class UnboundedKafkaReader + extends UnboundedReader> { + + private final UnboundedKafkaSource source; + private KafkaConsumer consumer; + + private final boolean isRawSource; // i.e. if key and value decoders are identity functions + private final long[] consumedOffsets; // consumed offset of each of the partitions, + // initialized to offset from checkpoint or -1. + + // --> index into consumerOffsets for this TopicPartition. + private final Map> offsetIndexMap; + + private ConsumerRecord curRecord; + private Instant curTimestamp; + + private Iterator> curBatch = Iterators.emptyIterator(); + + public UnboundedKafkaReader( + UnboundedKafkaSource source, + @Nullable KafkaCheckpointMark checkpointMark) { + this.source = source; + this.isRawSource = source.keyDecoderFn instanceof IdentityFn + && source.valueDecoderFn instanceof IdentityFn; + + consumedOffsets = new long[source.assignedPartitions.size()]; + + // a) verify that assigned and check-pointed partitions match + // b) set consumed offsets + if (checkpointMark != null) { + Preconditions.checkState( + checkpointMark.getPartitions().size() == source.assignedPartitions.size(), + "checkPointMark and assignedPartitions should match"); + // we could consider allowing a mismatch, though it is not expected in current Dataflow + + for (int i=0; i < consumedOffsets.length; i++) { + KafkaCheckpointMark.PartitionMark ckptMark = checkpointMark.getPartitions().get(i); + TopicPartition assigned = source.assignedPartitions.get(i); + + Preconditions.checkState(ckptMark.getTopicPartition().equals(assigned), + "checkpointed partition %s and assinged partition %s don't match at position %d", + ckptMark.getTopicPartition(), assigned, i); + + + consumedOffsets[i] = checkpointMark.getPartitions().get(i).getOffset(); + } + } else { + Arrays.fill(consumedOffsets, -1L); + } + + //make ImmutableMap(topic -> ImmutableMap(paritionId -> index in offsets array)) + offsetIndexMap = ImmutableMap.copyOf(IntStream + .range(0, source.assignedPartitions.size()) + .mapToObj(idx -> KV.of(source.assignedPartitions.get(idx), idx)) + .collect(Collectors.groupingBy(e -> e.getKey().topic(), + Collectors.toMap(e -> e.getKey().partition(), e -> e.getValue()))) + .entrySet() // Entry Map index>> + .stream() + .collect(Collectors.toMap(e -> e.getKey(), e -> ImmutableMap.copyOf(e.getValue())))); + } + + @Override + public boolean start() throws IOException { + + consumer = new KafkaConsumer<>(source.consumerConfig); + consumer.assign(source.assignedPartitions); + + // seek to offset if resuming + for(int i=0; i= 0) { + LOG.info("Reader: resuming %s at %d", partition, offset+1); + consumer.seek(partition, offset+1); + } else { + LOG.info("Reader: resuming from default offset for " + partition); + } + } + + curBatch = consumer.poll(10).iterator(); + + return curBatch.hasNext(); + } + + @SuppressWarnings("unchecked") // for rawRecord to curRecord cast below + @Override + public boolean advance() throws IOException { + while (true) { + if (curBatch.hasNext()) { + ConsumerRecord rawRecord = curBatch.next(); + + int idx = offsetIndexMap.get(rawRecord.topic()).get(rawRecord.partition()); + long consumedOffset = consumedOffsets[idx]; + + if (consumedOffsets[idx] >= 0 && rawRecord.offset() <= consumedOffset) { + // this can happen when compression is enabled in kafka + // should we check if the offset is way off from consumedOffset (say 1M more or less) + + LOG.info("ignoring already consumed offset %d for %s", rawRecord.offset(), + source.assignedPartitions.get(idx)); + // TODO: increment a counter? + + continue; + } else { + + // apply user decoders + if (isRawSource) { + // is shortcut this worth it? mostly not. + curRecord = (ConsumerRecord) rawRecord; + } else { + curRecord = new ConsumerRecord( + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + source.keyDecoderFn.apply(rawRecord.key()), + source.valueDecoderFn.apply(rawRecord.value())); + } + + curTimestamp = source.timestampFn.apply(curRecord); + consumedOffsets[idx] = rawRecord.offset(); + + return true; + } + } else { + // try to read next batch + curBatch = consumer.poll(10).iterator(); // what should the timeout be 0 or something large? + if (!curBatch.hasNext()) + return false; + } + } + } + + @Override + public Instant getWatermark() { + return source.timestampFn.apply(curRecord); + } + + @Override + public CheckpointMark getCheckpointMark() { + return new KafkaCheckpointMark(IntStream + .range(0, consumedOffsets.length) + .mapToObj(i -> new KafkaCheckpointMark.PartitionMark( + source.assignedPartitions.get(i), consumedOffsets[i])) + .collect(Collectors.toList())); + } + + @Override + public UnboundedSource, ?> getCurrentSource() { + return null; + } + + @Override + public ConsumerRecord getCurrent() throws NoSuchElementException { + // TODO: should we delay updating consumed offset till now? + return curRecord; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return curTimestamp; //TODO: how is this related to getWatermark(); + } + + @Override + public void close() throws IOException { + Closeables.closeQuietly(consumer); + } + } +} From af9b8871d65300fb988dfdbb569ab5991a7b2ae7 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 18 Feb 2016 13:32:19 -0800 Subject: [PATCH 03/72] remove java 1.7 restriction in pom --- contrib/kafka/pom.xml | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 94016c1f91..6004604bfe 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -41,16 +41,6 @@ - - org.apache.maven.plugins - maven-compiler-plugin - 3.2 - - 1.7 - 1.7 - - - org.apache.maven.plugins maven-checkstyle-plugin From 88767fd178bcd32cf101c527da62a600d78cf034 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 18 Feb 2016 16:36:28 -0800 Subject: [PATCH 04/72] Revert "remove java 1.7 restriction in pom" This reverts commit af9b8871d65300fb988dfdbb569ab5991a7b2ae7. --- contrib/kafka/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 6004604bfe..94016c1f91 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -41,6 +41,16 @@ + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + org.apache.maven.plugins maven-checkstyle-plugin From 464ad1e4d7d7d705fd5e07ee8049036bdf80bbc6 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 18 Feb 2016 16:54:25 -0800 Subject: [PATCH 05/72] simplify consumedOffset management with PartitionState --- contrib/kafka/pom.xml | 4 +- .../dataflow/contrib/kafka/KafkaSource.java | 136 +++++++++++------- 2 files changed, 86 insertions(+), 54 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 94016c1f91..4ebaaf2d50 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -46,8 +46,8 @@ maven-compiler-plugin 3.2 - 1.7 - 1.7 + 1.8 + 1.8 diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 552863fb25..2ca07a8387 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; -import java.util.AbstractMap; import java.util.Arrays; import java.util.Collection; import java.util.Iterator; @@ -52,7 +51,6 @@ import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.io.Closeables; @@ -133,7 +131,7 @@ private Builder() {} * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). */ public Builder withBootstrapServers(String bootstrapServers) { - return withConsumerConfig("bootstrap.servers", bootstrapServers); + return withConsumerProperty("bootstrap.servers", bootstrapServers); } /** @@ -148,13 +146,23 @@ public Builder withTopics(Collection topics) { * Set a {@KafkaConsumer} configuration properties. * @see ConsumerConfig */ - public Builder withConsumerConfig(String configKey, Object configValue) { + public Builder withConsumerProperty(String configKey, Object configValue) { Preconditions.checkArgument(ignoredConsumerProperties.containsKey(configKey), "No need to configure '%s'. %s", configKey, ignoredConsumerProperties.get(configKey)); consumerConfigBuilder.put(configKey, configValue); return this; } + /** + * Update consumer config properties. Note that this does not not discard already configurured. + * Same as invoking #withConsumerProperty() with each entry. + */ + public Builder withConsumerProperties(Map configToUpdate) { + configToUpdate.entrySet().stream().forEach( + e -> withConsumerProperty(e.getKey(), e.getValue())); + return this; + } + public Builder withKeyDecoderFn( SerializableFunction keyDecoderFn) { this.keyDecoderFn = keyDecoderFn; @@ -325,17 +333,40 @@ private static class UnboundedKafkaReader private final UnboundedKafkaSource source; private KafkaConsumer consumer; - private final boolean isRawSource; // i.e. if key and value decoders are identity functions - private final long[] consumedOffsets; // consumed offset of each of the partitions, - // initialized to offset from checkpoint or -1. + // maintains state of each assigned partition + + private static class PartitionState implements Iterator { + private final TopicPartition topicPartition; + + private Iterator> recordIter = Iterators.emptyIterator(); + private ConsumerRecord record = null; - // --> index into consumerOffsets for this TopicPartition. - private final Map> offsetIndexMap; + private long consumedOffset; + + PartitionState(TopicPartition partition, long offset) { + this.topicPartition = partition; + this.consumedOffset = offset; + } + + @Override + public boolean hasNext() { + return recordIter.hasNext(); + } + + @Override + public PartitionState next() { + record = recordIter.next(); + return this; + } + } + + private final boolean isRawSource; // i.e. if key and value decoders are identity functions + List partitionStates; private ConsumerRecord curRecord; private Instant curTimestamp; - private Iterator> curBatch = Iterators.emptyIterator(); + private Iterator curBatch = Iterators.emptyIterator(); public UnboundedKafkaReader( UnboundedKafkaSource source, @@ -344,7 +375,10 @@ public UnboundedKafkaReader( this.isRawSource = source.keyDecoderFn instanceof IdentityFn && source.valueDecoderFn instanceof IdentityFn; - consumedOffsets = new long[source.assignedPartitions.size()]; + partitionStates = ImmutableList.copyOf(source.assignedPartitions + .stream() + .map(tp -> new PartitionState(tp, -1L)) + .iterator()); // a) verify that assigned and check-pointed partitions match // b) set consumed offsets @@ -354,7 +388,7 @@ public UnboundedKafkaReader( "checkPointMark and assignedPartitions should match"); // we could consider allowing a mismatch, though it is not expected in current Dataflow - for (int i=0; i < consumedOffsets.length; i++) { + for (int i=0; i < source.assignedPartitions.size(); i++) { KafkaCheckpointMark.PartitionMark ckptMark = checkpointMark.getPartitions().get(i); TopicPartition assigned = source.assignedPartitions.get(i); @@ -363,21 +397,23 @@ public UnboundedKafkaReader( ckptMark.getTopicPartition(), assigned, i); - consumedOffsets[i] = checkpointMark.getPartitions().get(i).getOffset(); + partitionStates.get(i).consumedOffset = checkpointMark.getPartitions().get(i).getOffset(); } - } else { - Arrays.fill(consumedOffsets, -1L); } + } - //make ImmutableMap(topic -> ImmutableMap(paritionId -> index in offsets array)) - offsetIndexMap = ImmutableMap.copyOf(IntStream - .range(0, source.assignedPartitions.size()) - .mapToObj(idx -> KV.of(source.assignedPartitions.get(idx), idx)) - .collect(Collectors.groupingBy(e -> e.getKey().topic(), - Collectors.toMap(e -> e.getKey().partition(), e -> e.getValue()))) - .entrySet() // Entry Map index>> - .stream() - .collect(Collectors.toMap(e -> e.getKey(), e -> ImmutableMap.copyOf(e.getValue())))); + private void readNextBatch() { + // read one batch of records. one consumer.poll() + + ConsumerRecords records = consumer.poll(10); // what should the timeout be? + + partitionStates.stream().forEach(p -> { + p.recordIter = records.records(p.topicPartition).iterator(); + p.record = null; + }); + + curBatch = Iterators.concat(partitionStates.iterator()); + curRecord = null; } @Override @@ -386,20 +422,17 @@ public boolean start() throws IOException { consumer = new KafkaConsumer<>(source.consumerConfig); consumer.assign(source.assignedPartitions); - // seek to offset if resuming - for(int i=0; i= 0) { - LOG.info("Reader: resuming %s at %d", partition, offset+1); - consumer.seek(partition, offset+1); + // seek to next offset if consumedOffset is set + partitionStates.stream().forEach(p -> { + if (p.consumedOffset >= 0) { + LOG.info("Reader: resuming %s at %d", p.topicPartition, p.consumedOffset + 1); + consumer.seek(p.topicPartition, p.consumedOffset); } else { - LOG.info("Reader: resuming from default offset for " + partition); + LOG.info("Reader: resuming from default offset for " + p.topicPartition); } - } + }); - curBatch = consumer.poll(10).iterator(); + readNextBatch(); return curBatch.hasNext(); } @@ -409,26 +442,26 @@ public boolean start() throws IOException { public boolean advance() throws IOException { while (true) { if (curBatch.hasNext()) { - ConsumerRecord rawRecord = curBatch.next(); + PartitionState pState = curBatch.next(); - int idx = offsetIndexMap.get(rawRecord.topic()).get(rawRecord.partition()); - long consumedOffset = consumedOffsets[idx]; + ConsumerRecord rawRecord = pState.record; + long consumed = pState.consumedOffset; - if (consumedOffsets[idx] >= 0 && rawRecord.offset() <= consumedOffset) { - // this can happen when compression is enabled in kafka + if (consumed >= 0 && rawRecord.offset() <= consumed) { + // this can happen when compression is enabled in Kafka // should we check if the offset is way off from consumedOffset (say 1M more or less) + LOG.info("ignoring already consumed offset %d for %s", + rawRecord.offset(), pState.topicPartition); - LOG.info("ignoring already consumed offset %d for %s", rawRecord.offset(), - source.assignedPartitions.get(idx)); // TODO: increment a counter? - continue; + } else { // apply user decoders if (isRawSource) { // is shortcut this worth it? mostly not. - curRecord = (ConsumerRecord) rawRecord; + curRecord = (ConsumerRecord) pState.record; } else { curRecord = new ConsumerRecord( rawRecord.topic(), @@ -439,13 +472,13 @@ public boolean advance() throws IOException { } curTimestamp = source.timestampFn.apply(curRecord); - consumedOffsets[idx] = rawRecord.offset(); + pState.consumedOffset = rawRecord.offset(); return true; } } else { - // try to read next batch - curBatch = consumer.poll(10).iterator(); // what should the timeout be 0 or something large? + readNextBatch(); + if (!curBatch.hasNext()) return false; } @@ -459,16 +492,15 @@ public Instant getWatermark() { @Override public CheckpointMark getCheckpointMark() { - return new KafkaCheckpointMark(IntStream - .range(0, consumedOffsets.length) - .mapToObj(i -> new KafkaCheckpointMark.PartitionMark( - source.assignedPartitions.get(i), consumedOffsets[i])) + return new KafkaCheckpointMark(partitionStates + .stream() + .map(p -> new KafkaCheckpointMark.PartitionMark(p.topicPartition, p.consumedOffset)) .collect(Collectors.toList())); } @Override public UnboundedSource, ?> getCurrentSource() { - return null; + return source; } @Override From ed9f92a6e01ffdccff432b19420930eaa24593c3 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 18 Feb 2016 17:12:21 -0800 Subject: [PATCH 06/72] minor --- .../google/cloud/dataflow/contrib/kafka/KafkaSource.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 2ca07a8387..e09a74b980 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -45,7 +44,6 @@ import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -382,6 +380,7 @@ public UnboundedKafkaReader( // a) verify that assigned and check-pointed partitions match // b) set consumed offsets + if (checkpointMark != null) { Preconditions.checkState( checkpointMark.getPartitions().size() == source.assignedPartitions.size(), @@ -396,8 +395,7 @@ public UnboundedKafkaReader( "checkpointed partition %s and assinged partition %s don't match at position %d", ckptMark.getTopicPartition(), assigned, i); - - partitionStates.get(i).consumedOffset = checkpointMark.getPartitions().get(i).getOffset(); + partitionStates.get(i).consumedOffset = ckptMark.getOffset(); } } } From e42533b27bec320ef478e660b5eccebcccbcaa34 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 11:46:27 -0800 Subject: [PATCH 07/72] minor --- .../com/google/cloud/dataflow/contrib/kafka/KafkaSource.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index e09a74b980..005e33bb76 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -129,7 +129,7 @@ private Builder() {} * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). */ public Builder withBootstrapServers(String bootstrapServers) { - return withConsumerProperty("bootstrap.servers", bootstrapServers); + return withConsumerProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); } /** @@ -145,7 +145,7 @@ public Builder withTopics(Collection topics) { * @see ConsumerConfig */ public Builder withConsumerProperty(String configKey, Object configValue) { - Preconditions.checkArgument(ignoredConsumerProperties.containsKey(configKey), + Preconditions.checkArgument(!ignoredConsumerProperties.containsKey(configKey), "No need to configure '%s'. %s", configKey, ignoredConsumerProperties.get(configKey)); consumerConfigBuilder.put(configKey, configValue); return this; @@ -332,7 +332,6 @@ private static class UnboundedKafkaReader private KafkaConsumer consumer; // maintains state of each assigned partition - private static class PartitionState implements Iterator { private final TopicPartition topicPartition; From d419594328e466aa331a3a9760feec214acaad77 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 11:48:20 -0800 Subject: [PATCH 08/72] TopHashtagsExample (just does global count for now) --- .../contrib/kafka/TopHashtagsExample.java | 74 +++++++++++++++++++ 1 file changed, 74 insertions(+) create mode 100644 contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java new file mode 100644 index 0000000000..35bd75f496 --- /dev/null +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -0,0 +1,74 @@ +package com.google.cloud.dataflow.contrib.kafka; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import com.google.cloud.dataflow.sdk.options.Default; +import com.google.cloud.dataflow.sdk.options.Description; +import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.FlatMapElements; +import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; +import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; +import com.google.common.base.Charsets; +import com.google.common.collect.ImmutableList; + +/** + * Every minute, print top English hashtags over 10 last 10 minutes + * TODO: Move this out this directory. + */ +public class TopHashtagsExample { + private static final Logger LOG = LoggerFactory.getLogger(TopHashtagsExample.class); + + public static interface Options extends PipelineOptions { + @Description("Sliding window size, in minutes") + @Default.Integer(10) + Integer getSlidingWindowSize(); + void setSlidingWindowSize(Integer value); + + @Description("Trigger window size, in minutes") + @Default.Integer(1) + Integer getSlidingWindowPeriod(); + void setSlidingWindowPeriod(Integer value); + } + + public static void main(String args[]) { + + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + Pipeline pipeline = Pipeline.create(options); + + UnboundedSource, ?> kafkaSource = KafkaSource + .unboundedSourceBuilder() + .withBootstrapServers("localhost:9092") + .withTopics(ImmutableList.of("sample_tweets_json")) + .withKeyDecoderFn( bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) + .withValueDecoderFn(bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) + .build(); + + pipeline + .apply(Read.from(kafkaSource).named("sample_tweets")) + .apply(MapElements + ., Integer>via(r -> 1) + .withOutputType(new TypeDescriptor(){})) + .apply(Window.into(SlidingWindows + .of(Duration.standardMinutes(options.getSlidingWindowSize())) + .every(Duration.standardMinutes(options.getSlidingWindowPeriod())))) + .apply(Count.globally().withoutDefaults()) + .apply(FlatMapElements + .via(count -> { + LOG.info("Tweets in last 5 minutes : %d", count); + return ImmutableList.of();}) + .withOutputType(new TypeDescriptor(){})); + + pipeline.run(); + } +} From 74a27cc47bc4683dbc147846a857d464897083c6 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 11:54:46 -0800 Subject: [PATCH 09/72] set maxNumRecords to be able to run with DirectRunner --- .../cloud/dataflow/contrib/kafka/TopHashtagsExample.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 35bd75f496..a90f378d8c 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -55,7 +55,9 @@ public static void main(String args[]) { .build(); pipeline - .apply(Read.from(kafkaSource).named("sample_tweets")) + .apply(Read.from(kafkaSource) + .named("sample_tweets") + .withMaxNumRecords(10000)) // XXX work around for DirectRunner .apply(MapElements ., Integer>via(r -> 1) .withOutputType(new TypeDescriptor(){})) From 773ba2b5540ebcfdbeda00a01e2494db1f96191c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 15:24:52 -0800 Subject: [PATCH 10/72] add log4j dependency (temp) --- contrib/kafka/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 4ebaaf2d50..6fb0222891 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -147,6 +147,18 @@ [0.9,) + + + log4j + log4j + 1.2.17 + + + org.slf4j + slf4j-log4j12 + 1.7.16 + + junit From 5990e93fc817f23367f2f9d364ba7566e7cc805e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 15:25:17 -0800 Subject: [PATCH 11/72] some fixes and some temp code --- .../dataflow/contrib/kafka/KafkaSource.java | 121 ++++++++++++++++-- .../contrib/kafka/TopHashtagsExample.java | 5 +- 2 files changed, 112 insertions(+), 14 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 005e33bb76..ea055c1b60 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -17,6 +17,8 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -37,13 +39,17 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.util.CloudObject; +import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -104,12 +110,7 @@ public static class Builder { private SerializableFunction, Instant> timestampFn = new NowTimestampFn>(); // default processing timestamp - private ImmutableMap.Builder consumerConfigBuilder = ImmutableMap - .builder() - .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()) - .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()) - .put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest") // default to latest offset when last offset is unknown. - .put("enable.auto.commit", false); // disable auto commit (may be enabled by the user) + private Map mutableConsumerConfig = Maps.newHashMap(); /** * set of properties that are not required or don't make sense @@ -123,7 +124,14 @@ public static class Builder { ); - private Builder() {} + private Builder() { + // set config defaults + mutableConsumerConfig.putAll(ImmutableMap.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", // default to latest offset when last offset is unknown. + "enable.auto.commit", false)); // disable auto commit (may be enabled by the user) + } /** * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). @@ -147,7 +155,7 @@ public Builder withTopics(Collection topics) { public Builder withConsumerProperty(String configKey, Object configValue) { Preconditions.checkArgument(!ignoredConsumerProperties.containsKey(configKey), "No need to configure '%s'. %s", configKey, ignoredConsumerProperties.get(configKey)); - consumerConfigBuilder.put(configKey, configValue); + mutableConsumerConfig.put(configKey, configValue); return this; } @@ -185,7 +193,7 @@ public Builder withTimestampFn( public UnboundedSource, KafkaCheckpointMark> build() { - ImmutableMap consumerConfig = consumerConfigBuilder.build(); + ImmutableMap consumerConfig = ImmutableMap.copyOf(mutableConsumerConfig); Preconditions.checkNotNull( consumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), @@ -232,13 +240,30 @@ public UnboundedKafkaSource( this.keyDecoderFn = keyDecoderFn; this.valueDecoderFn = valueDecoderFn; this.timestampFn = timestampFn; - this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); + + if (assignedPartitions.size() == 0) { + // XXX Temp workaround for DirectRunner. Fetch partitions. + KafkaConsumer consumer = new KafkaConsumer(consumerConfig); + this.assignedPartitions = topics + .stream() + .flatMap(topic -> consumer.partitionsFor(topic).stream()) + .map(partInfo -> new TopicPartition(partInfo.topic(), partInfo.partition())) + .sorted((p1, p2) -> ComparisonChain.start() // sort by + .compare(p1.partition(), p2.partition()) + .compare(p1.topic(), p2.topic()) + .result()) + .collect(Collectors.toList()); + consumer.close(); + } else { + this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); + } } @Override public List, KafkaCheckpointMark>> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { + // XXX : not invoked by DirectRunner // XXX : I was checking various Java 8 streams and collectors.. thats is the reason for heavy use them here :) KafkaConsumer consumer = new KafkaConsumer(consumerConfig); @@ -321,7 +346,75 @@ public void validate() { @Override public Coder> getDefaultOutputCoder() { // no coder required. user explicitly provides functions to decode key and value - return null; + // XXX Source needs to provide OutputCoder? + return new Coder>() { + + @Override + public void encode(ConsumerRecord value, OutputStream outStream, + com.google.cloud.dataflow.sdk.coders.Coder.Context context) + throws CoderException, IOException { + } + + @Override + public ConsumerRecord decode(InputStream inStream, + com.google.cloud.dataflow.sdk.coders.Coder.Context context) + throws CoderException, IOException { + return null; + } + + @Override + public List> getCoderArguments() { + return null; + } + + @Override + public CloudObject asCloudObject() { + return null; + } + + @Override + public void verifyDeterministic() + throws com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException { + } + + @Override + public boolean consistentWithEquals() { + return false; + } + + @Override + public Object structuralValue(ConsumerRecord value) throws Exception { + return null; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(ConsumerRecord value, + com.google.cloud.dataflow.sdk.coders.Coder.Context context) { + // TODO Auto-generated method stub + return false; + } + + @Override + public void registerByteSizeObserver(ConsumerRecord value, + ElementByteSizeObserver observer, + com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws Exception { + // TODO Auto-generated method stub + + } + + @Override + public String getEncodingId() { + // TODO Auto-generated method stub + return null; + } + + @Override + public Collection getAllowedEncodings() { + // TODO Auto-generated method stub + return null; + } + + }; } } @@ -404,6 +497,10 @@ private void readNextBatch() { ConsumerRecords records = consumer.poll(10); // what should the timeout be? + if (records.count() > 0) { + LOG.info("XXX : read " + records.count() + " records"); + } + partitionStates.stream().forEach(p -> { p.recordIter = records.records(p.topicPartition).iterator(); p.record = null; @@ -425,7 +522,7 @@ public boolean start() throws IOException { LOG.info("Reader: resuming %s at %d", p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset); } else { - LOG.info("Reader: resuming from default offset for " + p.topicPartition); + LOG.info("Reader: resuming from default offset for %s", p.topicPartition); } }); diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index a90f378d8c..07d46fcd18 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -50,6 +50,7 @@ public static void main(String args[]) { .unboundedSourceBuilder() .withBootstrapServers("localhost:9092") .withTopics(ImmutableList.of("sample_tweets_json")) + //.withConsumerProperty("auto.offset.reset", "earliest") // XXX Temp .withKeyDecoderFn( bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) .withValueDecoderFn(bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) .build(); @@ -57,13 +58,13 @@ public static void main(String args[]) { pipeline .apply(Read.from(kafkaSource) .named("sample_tweets") - .withMaxNumRecords(10000)) // XXX work around for DirectRunner + .withMaxNumRecords(1000)) // XXX work around for DirectRunner .apply(MapElements ., Integer>via(r -> 1) .withOutputType(new TypeDescriptor(){})) .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(options.getSlidingWindowSize())) - .every(Duration.standardMinutes(options.getSlidingWindowPeriod())))) + .every(Duration.standardSeconds(options.getSlidingWindowPeriod())))) .apply(Count.globally().withoutDefaults()) .apply(FlatMapElements .via(count -> { From 7d33c59daa9bc32cf83e13997388d791bacb6bfa Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 19 Feb 2016 16:15:52 -0800 Subject: [PATCH 12/72] Add KafkaRecord (a Serializable version of kafka.ConsumerRecord) --- .../dataflow/contrib/kafka/KafkaSource.java | 128 ++++-------------- .../contrib/kafka/TopHashtagsExample.java | 7 +- 2 files changed, 28 insertions(+), 107 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index ea055c1b60..47cd7940dc 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -17,8 +17,6 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -42,14 +40,13 @@ import com.google.api.client.util.Maps; import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.cloud.dataflow.sdk.util.CloudObject; -import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver; +import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -107,8 +104,8 @@ public static class Builder { // future: let users specify subset of partitions to read private SerializableFunction keyDecoderFn; private SerializableFunction valueDecoderFn; - private SerializableFunction, Instant> timestampFn = - new NowTimestampFn>(); // default processing timestamp + private SerializableFunction, Instant> timestampFn = + new NowTimestampFn>(); // default processing timestamp private Map mutableConsumerConfig = Maps.newHashMap(); @@ -186,12 +183,12 @@ public Builder withValueDecoderFn( * by {@UnboundedReader#advance()} */ public Builder withTimestampFn( - SerializableFunction, Instant> timestampFn) { + SerializableFunction, Instant> timestampFn) { this.timestampFn = timestampFn; return this; } - public UnboundedSource, KafkaCheckpointMark> build() { + public UnboundedSource, KafkaCheckpointMark> build() { ImmutableMap consumerConfig = ImmutableMap.copyOf(mutableConsumerConfig); @@ -218,13 +215,13 @@ public UnboundedSource, KafkaCheckpointMark> build() { private KafkaSource() {} private static class UnboundedKafkaSource - extends UnboundedSource, KafkaCheckpointMark> { + extends UnboundedSource, KafkaCheckpointMark> { private final ImmutableMap consumerConfig; private final List topics; private final SerializableFunction keyDecoderFn; private final SerializableFunction valueDecoderFn; - private final SerializableFunction, Instant> timestampFn; + private final SerializableFunction, Instant> timestampFn; private final List assignedPartitions; public UnboundedKafkaSource( @@ -232,7 +229,7 @@ public UnboundedKafkaSource( List topics, SerializableFunction keyDecoderFn, SerializableFunction valueDecoderFn, - SerializableFunction, Instant> timestampFn, + SerializableFunction, Instant> timestampFn, List assignedPartitions) { this.consumerConfig = consumerConfig; @@ -260,7 +257,7 @@ public UnboundedKafkaSource( } @Override - public List, KafkaCheckpointMark>> generateInitialSplits( + public List, KafkaCheckpointMark>> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { // XXX : not invoked by DirectRunner @@ -322,7 +319,7 @@ public List, KafkaCheckpointMark> } @Override - public UnboundedReader> createReader( + public UnboundedReader> createReader( PipelineOptions options, KafkaCheckpointMark checkpointMark) { return new UnboundedKafkaReader(this, checkpointMark); @@ -344,82 +341,15 @@ public void validate() { } @Override - public Coder> getDefaultOutputCoder() { + public Coder> getDefaultOutputCoder() { // no coder required. user explicitly provides functions to decode key and value // XXX Source needs to provide OutputCoder? - return new Coder>() { - - @Override - public void encode(ConsumerRecord value, OutputStream outStream, - com.google.cloud.dataflow.sdk.coders.Coder.Context context) - throws CoderException, IOException { - } - - @Override - public ConsumerRecord decode(InputStream inStream, - com.google.cloud.dataflow.sdk.coders.Coder.Context context) - throws CoderException, IOException { - return null; - } - - @Override - public List> getCoderArguments() { - return null; - } - - @Override - public CloudObject asCloudObject() { - return null; - } - - @Override - public void verifyDeterministic() - throws com.google.cloud.dataflow.sdk.coders.Coder.NonDeterministicException { - } - - @Override - public boolean consistentWithEquals() { - return false; - } - - @Override - public Object structuralValue(ConsumerRecord value) throws Exception { - return null; - } - - @Override - public boolean isRegisterByteSizeObserverCheap(ConsumerRecord value, - com.google.cloud.dataflow.sdk.coders.Coder.Context context) { - // TODO Auto-generated method stub - return false; - } - - @Override - public void registerByteSizeObserver(ConsumerRecord value, - ElementByteSizeObserver observer, - com.google.cloud.dataflow.sdk.coders.Coder.Context context) throws Exception { - // TODO Auto-generated method stub - - } - - @Override - public String getEncodingId() { - // TODO Auto-generated method stub - return null; - } - - @Override - public Collection getAllowedEncodings() { - // TODO Auto-generated method stub - return null; - } - - }; + return SerializableCoder.of(new TypeDescriptor>() {}); } } private static class UnboundedKafkaReader - extends UnboundedReader> { + extends UnboundedReader> { private final UnboundedKafkaSource source; private KafkaConsumer consumer; @@ -450,10 +380,9 @@ record = recordIter.next(); } } - private final boolean isRawSource; // i.e. if key and value decoders are identity functions - List partitionStates; + private List partitionStates; - private ConsumerRecord curRecord; + private KafkaRecord curRecord; private Instant curTimestamp; private Iterator curBatch = Iterators.emptyIterator(); @@ -461,9 +390,8 @@ record = recordIter.next(); public UnboundedKafkaReader( UnboundedKafkaSource source, @Nullable KafkaCheckpointMark checkpointMark) { + this.source = source; - this.isRawSource = source.keyDecoderFn instanceof IdentityFn - && source.valueDecoderFn instanceof IdentityFn; partitionStates = ImmutableList.copyOf(source.assignedPartitions .stream() @@ -531,7 +459,6 @@ public boolean start() throws IOException { return curBatch.hasNext(); } - @SuppressWarnings("unchecked") // for rawRecord to curRecord cast below @Override public boolean advance() throws IOException { while (true) { @@ -553,17 +480,12 @@ public boolean advance() throws IOException { } else { // apply user decoders - if (isRawSource) { - // is shortcut this worth it? mostly not. - curRecord = (ConsumerRecord) pState.record; - } else { - curRecord = new ConsumerRecord( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - source.keyDecoderFn.apply(rawRecord.key()), - source.valueDecoderFn.apply(rawRecord.value())); - } + curRecord = new KafkaRecord( + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + source.keyDecoderFn.apply(rawRecord.key()), + source.valueDecoderFn.apply(rawRecord.value())); curTimestamp = source.timestampFn.apply(curRecord); pState.consumedOffset = rawRecord.offset(); @@ -593,12 +515,12 @@ public CheckpointMark getCheckpointMark() { } @Override - public UnboundedSource, ?> getCurrentSource() { + public UnboundedSource, ?> getCurrentSource() { return source; } @Override - public ConsumerRecord getCurrent() throws NoSuchElementException { + public KafkaRecord getCurrent() throws NoSuchElementException { // TODO: should we delay updating consumed offset till now? return curRecord; } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 07d46fcd18..79bc4193fc 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -1,6 +1,5 @@ package com.google.cloud.dataflow.contrib.kafka; -import org.apache.kafka.clients.consumer.ConsumerRecord; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,7 +45,7 @@ public static void main(String args[]) { Pipeline pipeline = Pipeline.create(options); - UnboundedSource, ?> kafkaSource = KafkaSource + UnboundedSource, ?> kafkaSource = KafkaSource .unboundedSourceBuilder() .withBootstrapServers("localhost:9092") .withTopics(ImmutableList.of("sample_tweets_json")) @@ -60,7 +59,7 @@ public static void main(String args[]) { .named("sample_tweets") .withMaxNumRecords(1000)) // XXX work around for DirectRunner .apply(MapElements - ., Integer>via(r -> 1) + ., Integer>via(r -> 1) .withOutputType(new TypeDescriptor(){})) .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(options.getSlidingWindowSize())) @@ -68,7 +67,7 @@ public static void main(String args[]) { .apply(Count.globally().withoutDefaults()) .apply(FlatMapElements .via(count -> { - LOG.info("Tweets in last 5 minutes : %d", count); + LOG.info("Tweets in last 5 minutes : " + count); return ImmutableList.of();}) .withOutputType(new TypeDescriptor(){})); From 3a1dc97ba1261202f6ec9918850b438c1e2ee214 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 23 Feb 2016 09:45:51 -0800 Subject: [PATCH 13/72] Builder for ValueSource --- contrib/kafka/pom.xml | 8 + .../dataflow/contrib/kafka/KafkaSource.java | 164 ++++++++++++++++-- 2 files changed, 162 insertions(+), 10 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 6fb0222891..4a389ce4a6 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -147,6 +147,14 @@ [0.9,) + + + org.apache.kafka + kafka_2.11 + 0.9.0.0 + provided + + log4j diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 47cd7940dc..de6db3e4e1 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -17,6 +17,7 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; +import java.io.Serializable; import java.util.Collection; import java.util.Iterator; import java.util.List; @@ -64,12 +65,9 @@ public class KafkaSource { private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); - /* TODO: - * - abstract out kafka interactions - * - should we let user specify key and value deserializers in kafka. I don't think so. key - * matters to Kafka only at the producer (hashCode is used for partition). the consumer does - * not care. We could use DataFlow coder or SerializableFunction. - * - leave a comment about optionally storing + /* TODO: Overall todos: + * - javadoc at many places + * - confirm non-blocking behavior in advance() */ private static class IdentityFn implements SerializableFunction { @@ -98,6 +96,16 @@ public static Builder unboundedByteSourceBuilder() { .withValueDecoderFn(new IdentityFn()); } + /** + * Similar to {@link #unboundedSourceBuilder()}, except the the source strips KafkaRecord wrapper + * and returns just the value. + */ + public static ValueSourceBuilder unboundedValueSourceBuilder() { + return new ValueSourceBuilder( + new Builder() + .withKeyDecoderFn(new IdentityFn())); + } + public static class Builder { private List topics; @@ -257,7 +265,7 @@ public UnboundedKafkaSource( } @Override - public List, KafkaCheckpointMark>> generateInitialSplits( + public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { // XXX : not invoked by DirectRunner @@ -319,9 +327,8 @@ public List, KafkaCheckpointMark>> g } @Override - public UnboundedReader> createReader( - PipelineOptions options, - KafkaCheckpointMark checkpointMark) { + public UnboundedKafkaReader createReader(PipelineOptions options, + KafkaCheckpointMark checkpointMark) { return new UnboundedKafkaReader(this, checkpointMark); } @@ -535,4 +542,141 @@ public void close() throws IOException { Closeables.closeQuietly(consumer); } } + + // Builder, Source, Reader wrappers when user is only interested in Value in KafkaRecord : + + public static class ValueSourceBuilder { + // TODO : remove 'extends Serializable' restriction or improve it to just require a Coder + + private Builder underlying; + + private ValueSourceBuilder(Builder underlying) { + this.underlying = underlying; + } + + public ValueSourceBuilder withBootstrapServers(String bootstrapServers) { + return new ValueSourceBuilder(underlying.withBootstrapServers(bootstrapServers)); + } + + public ValueSourceBuilder withTopics(Collection topics) { + return new ValueSourceBuilder(underlying.withTopics(topics)); + } + + public ValueSourceBuilder withConsumerProperty(String configKey, Object configValue) { + return new ValueSourceBuilder(underlying.withConsumerProperty(configKey, configValue)); + } + + public ValueSourceBuilder withConsumerProperties(Map configToUpdate) { + return new ValueSourceBuilder(underlying.withConsumerProperties(configToUpdate)); + } + + public ValueSourceBuilder withValueDecoderFn(SerializableFunction valueDecoderFn) { + return new ValueSourceBuilder(underlying.withValueDecoderFn(valueDecoderFn)); + } + + public ValueSourceBuilder withTimestampFn(SerializableFunction timestampFn) { + return new ValueSourceBuilder( + underlying.withTimestampFn(record -> timestampFn.apply(record.getValue()))); + } + + public UnboundedSource build() { + return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); + } + } + + /** + * Usually the users are only interested in value in KafkaRecord. This is a convenient class + * to strip out other fields in KafkaRecord returned by UnboundedKafkaValueSource + */ + private static class UnboundedKafkaValueSource extends UnboundedSource { + + private final UnboundedKafkaSource underlying; + + public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { + this.underlying = underlying; + } + + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + return underlying + .generateInitialSplits(desiredNumSplits, options) + .stream() + .map(s -> new UnboundedKafkaValueSource(s)) + .collect(Collectors.toList()); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { + return new UnboundedKafkaValueReader(this, underlying.createReader(options, checkpointMark)); + } + + @Override + public Coder getCheckpointMarkCoder() { + return underlying.getCheckpointMarkCoder(); + } + + @Override + public void validate() { + underlying.validate(); + } + + @Override + public Coder getDefaultOutputCoder() { + return SerializableCoder.of(new TypeDescriptor() {}); + } + } + + private static class UnboundedKafkaValueReader extends UnboundedReader { + + private final UnboundedKafkaValueSource source; + private final UnboundedKafkaReader underlying; + + public UnboundedKafkaValueReader(UnboundedKafkaValueSource source, + UnboundedKafkaReader underlying) { + this.source = source; + this.underlying = underlying; + } + + @Override + public boolean start() throws IOException { + return underlying.start(); + } + + @Override + public boolean advance() throws IOException { + return underlying.advance(); + } + + @Override + public Instant getWatermark() { + return underlying.getWatermark(); + } + + @Override + public CheckpointMark getCheckpointMark() { + return underlying.getCheckpointMark(); + } + + @Override + public UnboundedKafkaValueSource getCurrentSource() { + return source; + } + + @Override + public T getCurrent() throws NoSuchElementException { + return underlying.getCurrent().getValue(); + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return underlying.getCurrentTimestamp(); + } + + @Override + public void close() throws IOException { + underlying.close(); + } + } + } From 8580ed2b97724badb93692f70e2617c46a499121 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 23 Feb 2016 09:49:05 -0800 Subject: [PATCH 14/72] make identityFn a static val --- .../cloud/dataflow/contrib/kafka/KafkaSource.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index de6db3e4e1..88d09041d9 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -70,11 +70,7 @@ public class KafkaSource { * - confirm non-blocking behavior in advance() */ - private static class IdentityFn implements SerializableFunction { - public T apply(T input) { - return input; - } - } + private static SerializableFunction identityFn = bytes -> bytes; /** * A function that returns {@link Instant#now} as the timestamp for each generated element. @@ -92,8 +88,8 @@ public static Builder unboundedSourceBuilder() { public static Builder unboundedByteSourceBuilder() { return new Builder() - .withKeyDecoderFn(new IdentityFn()) - .withValueDecoderFn(new IdentityFn()); + .withKeyDecoderFn(identityFn) + .withValueDecoderFn(identityFn); } /** @@ -103,7 +99,7 @@ public static Builder unboundedByteSourceBuilder() { public static ValueSourceBuilder unboundedValueSourceBuilder() { return new ValueSourceBuilder( new Builder() - .withKeyDecoderFn(new IdentityFn())); + .withKeyDecoderFn(identityFn)); } public static class Builder { From a00625d4d4d095c9098640d4a358c8a911e9e5f3 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 23 Feb 2016 10:08:41 -0800 Subject: [PATCH 15/72] remove full kafka depenency (was used for old consumer) --- contrib/kafka/pom.xml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 4a389ce4a6..6fb0222891 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -147,14 +147,6 @@ [0.9,) - - - org.apache.kafka - kafka_2.11 - 0.9.0.0 - provided - - log4j From 7e8393ddc0dfd4c8c8bc1ffba7317cd897ec3c9f Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Wed, 24 Feb 2016 14:24:06 -0800 Subject: [PATCH 16/72] serialization fixes and update example to track top hashtags --- contrib/kafka/pom.xml | 10 +-- .../contrib/kafka/KafkaCheckpointMark.java | 21 ++---- .../dataflow/contrib/kafka/KafkaSource.java | 23 +++---- .../contrib/kafka/TopHashtagsExample.java | 65 ++++++++++++++----- 4 files changed, 69 insertions(+), 50 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 6fb0222891..a5fdc2e30f 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -148,16 +148,16 @@ - - log4j - log4j - 1.2.17 - org.slf4j slf4j-log4j12 1.7.16 + + log4j + log4j + 1.2.17 + diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java index eff1b03b9c..e57c941493 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java @@ -1,12 +1,13 @@ package com.google.cloud.dataflow.contrib.kafka; import java.io.IOException; +import java.io.Serializable; import java.util.List; import org.apache.kafka.common.TopicPartition; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.DefaultCoder; +import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; /** @@ -15,8 +16,8 @@ * * @author rangadi */ -@DefaultCoder(AvroCoder.class) -public class KafkaCheckpointMark implements UnboundedSource.CheckpointMark { +@DefaultCoder(SerializableCoder.class) +public class KafkaCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { private final List partitions; @@ -24,11 +25,6 @@ public KafkaCheckpointMark(List partitions) { this.partitions = partitions; } - @SuppressWarnings("unused") // for AvroCoder - private KafkaCheckpointMark() { - partitions = null; - } - public List getPartitions() { return partitions; } @@ -46,17 +42,10 @@ public void finalizeCheckpoint() throws IOException { */ } - public static class PartitionMark { + public static class PartitionMark implements Serializable { private final TopicPartition topicPartition; private final long offset; - /** - * TODO - * @param topic - * @param partition - * @param offset - * @param consumed - */ public PartitionMark(TopicPartition topicPartition, long offset) { this.topicPartition = topicPartition; this.offset = offset; diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 88d09041d9..b6473eb04a 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -243,8 +243,9 @@ public UnboundedKafkaSource( this.timestampFn = timestampFn; if (assignedPartitions.size() == 0) { - // XXX Temp workaround for DirectRunner. Fetch partitions. + // XXX workaround for DirectRunner. Fetch partitions since it does not call generateInitialSplits() KafkaConsumer consumer = new KafkaConsumer(consumerConfig); + LOG.info("XXX local runner hack"); this.assignedPartitions = topics .stream() .flatMap(topic -> consumer.partitionsFor(topic).stream()) @@ -264,8 +265,7 @@ public UnboundedKafkaSource( public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { - // XXX : not invoked by DirectRunner - // XXX : I was checking various Java 8 streams and collectors.. thats is the reason for heavy use them here :) + LOG.info("XXX generateInitialSplits()"); KafkaConsumer consumer = new KafkaConsumer(consumerConfig); @@ -301,16 +301,15 @@ public List> generateInitialSplits( // create a new source for each split with the assigned partitions for the split return IntStream.range(0, numSplits) .mapToObj(split -> { - List assignedToSplit = assignments.get(split) .stream() .map(i -> partitions.get(i)) .collect(Collectors.toList()); - LOG.info("Partitions assigned for split %d : %s", + LOG.info("Partitions assigned for split {} : {}", split, Joiner.on(",").join(assignedToSplit)); - // copy of 'this', except for assignedPartitions, which is replaced by assignedToSplit + // copy of 'this' with assignedPartitions replaced with assignedToSplit return new UnboundedKafkaSource( this.consumerConfig, this.topics, @@ -345,7 +344,7 @@ public void validate() { @Override public Coder> getDefaultOutputCoder() { - // no coder required. user explicitly provides functions to decode key and value + // no coder is logically needed. user explicitly provides functions to decode key and value // XXX Source needs to provide OutputCoder? return SerializableCoder.of(new TypeDescriptor>() {}); } @@ -428,9 +427,7 @@ private void readNextBatch() { ConsumerRecords records = consumer.poll(10); // what should the timeout be? - if (records.count() > 0) { - LOG.info("XXX : read " + records.count() + " records"); - } + // TODO: increment a counter or stat (for histogram) by records.count()? partitionStates.stream().forEach(p -> { p.recordIter = records.records(p.topicPartition).iterator(); @@ -450,10 +447,10 @@ public boolean start() throws IOException { // seek to next offset if consumedOffset is set partitionStates.stream().forEach(p -> { if (p.consumedOffset >= 0) { - LOG.info("Reader: resuming %s at %d", p.topicPartition, p.consumedOffset + 1); + LOG.info("Reader: resuming {} at {}", p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset); } else { - LOG.info("Reader: resuming from default offset for %s", p.topicPartition); + LOG.info("Reader: resuming from default offset for {}", p.topicPartition); } }); @@ -474,7 +471,7 @@ public boolean advance() throws IOException { if (consumed >= 0 && rawRecord.offset() <= consumed) { // this can happen when compression is enabled in Kafka // should we check if the offset is way off from consumedOffset (say 1M more or less) - LOG.info("ignoring already consumed offset %d for %s", + LOG.info("ignoring already consumed offset {} for {}", rawRecord.offset(), pState.topicPartition); // TODO: increment a counter? diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 79bc4193fc..2de9bd3e54 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -1,9 +1,13 @@ package com.google.cloud.dataflow.contrib.kafka; +import java.util.List; + import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.UnboundedSource; @@ -12,10 +16,13 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; import com.google.cloud.dataflow.sdk.transforms.Count; +import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.FlatMapElements; -import com.google.cloud.dataflow.sdk.transforms.MapElements; +import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.Top; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; +import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; @@ -37,6 +44,31 @@ public static interface Options extends PipelineOptions { @Default.Integer(1) Integer getSlidingWindowPeriod(); void setSlidingWindowPeriod(Integer value); + + @Description("Bootstarp Server(s) for Kafka") + String getBootstrapServers(); + void setBootstrapServers(String servers); + + @Description("Num of Top hashtags") + @Default.Integer(10) + Integer getNumTopHashtags(); + void setNumTopHashtags(Integer count); + } + + /** + * Emit each of the hashtags in tweet json + */ + static class ExtractHashtagsFn extends DoFn { + private final ObjectMapper jsonMapper = new ObjectMapper(); + + @Override + public void processElement(ProcessContext ctx) throws Exception { + for (JsonNode hashtag : jsonMapper.readTree(ctx.element()) + .with("entities") + .withArray("hashtags")) { + ctx.output(hashtag.get("text").asText()); + } + } } public static void main(String args[]) { @@ -45,29 +77,30 @@ public static void main(String args[]) { Pipeline pipeline = Pipeline.create(options); - UnboundedSource, ?> kafkaSource = KafkaSource - .unboundedSourceBuilder() - .withBootstrapServers("localhost:9092") + final int windowSize = options.getSlidingWindowSize(); + final int windowPeriod = options.getSlidingWindowPeriod(); + + UnboundedSource kafkaSource = KafkaSource + .unboundedValueSourceBuilder() + .withBootstrapServers(options.getBootstrapServers()) .withTopics(ImmutableList.of("sample_tweets_json")) //.withConsumerProperty("auto.offset.reset", "earliest") // XXX Temp - .withKeyDecoderFn( bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) .withValueDecoderFn(bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) .build(); pipeline .apply(Read.from(kafkaSource) - .named("sample_tweets") - .withMaxNumRecords(1000)) // XXX work around for DirectRunner - .apply(MapElements - ., Integer>via(r -> 1) - .withOutputType(new TypeDescriptor(){})) - .apply(Window.into(SlidingWindows - .of(Duration.standardMinutes(options.getSlidingWindowSize())) - .every(Duration.standardSeconds(options.getSlidingWindowPeriod())))) - .apply(Count.globally().withoutDefaults()) + //.withMaxNumRecords(1000)// needed for local runner + .named("sample_tweets")) + .apply(ParDo.of(new ExtractHashtagsFn())) + .apply(Window.into(SlidingWindows + .of(Duration.standardMinutes(windowSize)) + .every(Duration.standardSeconds(windowPeriod)))) + .apply(Count.perElement()) + .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()).withoutDefaults()) .apply(FlatMapElements - .via(count -> { - LOG.info("Tweets in last 5 minutes : " + count); + .via((List> top) -> { + LOG.info("Top Hashtags in {} minutes : {}", windowSize, top); return ImmutableList.of();}) .withOutputType(new TypeDescriptor(){})); From 49fd42f051df35f711ac35a8bd24fe1ab4860574 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Wed, 24 Feb 2016 15:04:31 -0800 Subject: [PATCH 17/72] serializer fix --- .../com/google/cloud/dataflow/contrib/kafka/KafkaSource.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index b6473eb04a..5a9d55491d 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -39,7 +39,6 @@ import org.slf4j.LoggerFactory; import com.google.api.client.util.Maps; -import com.google.cloud.dataflow.sdk.coders.AvroCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; @@ -329,7 +328,7 @@ public UnboundedKafkaReader createReader(PipelineOptions options, @Override public Coder getCheckpointMarkCoder() { - return AvroCoder.of(KafkaCheckpointMark.class); + return SerializableCoder.of(KafkaCheckpointMark.class); } @Override From a86f11e21297025dc25862cd910a409122640dab Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Wed, 24 Feb 2016 15:22:54 -0800 Subject: [PATCH 18/72] offset fix --- .../com/google/cloud/dataflow/contrib/kafka/KafkaSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 5a9d55491d..3d7b541881 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -447,7 +447,7 @@ public boolean start() throws IOException { partitionStates.stream().forEach(p -> { if (p.consumedOffset >= 0) { LOG.info("Reader: resuming {} at {}", p.topicPartition, p.consumedOffset + 1); - consumer.seek(p.topicPartition, p.consumedOffset); + consumer.seek(p.topicPartition, p.consumedOffset + 1); } else { LOG.info("Reader: resuming from default offset for {}", p.topicPartition); } From f8fb50d71856f3a54b09b0d444c5b8dab78b4376 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Wed, 24 Feb 2016 23:33:56 -0800 Subject: [PATCH 19/72] testing with timestampFn. getWatermark() odd behaviour --- .../dataflow/contrib/kafka/KafkaSource.java | 59 +++++++------------ .../contrib/kafka/TopHashtagsExample.java | 37 +++++++++--- 2 files changed, 51 insertions(+), 45 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 3d7b541881..743ae69a19 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -34,6 +34,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,16 +72,6 @@ public class KafkaSource { private static SerializableFunction identityFn = bytes -> bytes; - /** - * A function that returns {@link Instant#now} as the timestamp for each generated element. - */ - private static class NowTimestampFn implements SerializableFunction { - @Override - public Instant apply(T input) { - return Instant.now(); - } - } - public static Builder unboundedSourceBuilder() { return new Builder(); } @@ -107,8 +98,7 @@ public static class Builder { // future: let users specify subset of partitions to read private SerializableFunction keyDecoderFn; private SerializableFunction valueDecoderFn; - private SerializableFunction, Instant> timestampFn = - new NowTimestampFn>(); // default processing timestamp + private SerializableFunction, Instant> timestampFn = input -> Instant.now(); private Map mutableConsumerConfig = Maps.newHashMap(); @@ -240,32 +230,13 @@ public UnboundedKafkaSource( this.keyDecoderFn = keyDecoderFn; this.valueDecoderFn = valueDecoderFn; this.timestampFn = timestampFn; - - if (assignedPartitions.size() == 0) { - // XXX workaround for DirectRunner. Fetch partitions since it does not call generateInitialSplits() - KafkaConsumer consumer = new KafkaConsumer(consumerConfig); - LOG.info("XXX local runner hack"); - this.assignedPartitions = topics - .stream() - .flatMap(topic -> consumer.partitionsFor(topic).stream()) - .map(partInfo -> new TopicPartition(partInfo.topic(), partInfo.partition())) - .sorted((p1, p2) -> ComparisonChain.start() // sort by - .compare(p1.partition(), p2.partition()) - .compare(p1.topic(), p2.topic()) - .result()) - .collect(Collectors.toList()); - consumer.close(); - } else { - this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); - } + this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); } @Override public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { - LOG.info("XXX generateInitialSplits()"); - KafkaConsumer consumer = new KafkaConsumer(consumerConfig); List partitions; @@ -344,7 +315,7 @@ public void validate() { @Override public Coder> getDefaultOutputCoder() { // no coder is logically needed. user explicitly provides functions to decode key and value - // XXX Source needs to provide OutputCoder? + // sdk requires a Coder. return SerializableCoder.of(new TypeDescriptor>() {}); } } @@ -426,7 +397,7 @@ private void readNextBatch() { ConsumerRecords records = consumer.poll(10); // what should the timeout be? - // TODO: increment a counter or stat (for histogram) by records.count()? + // increment a counter or stat? partitionStates.stream().forEach(p -> { p.recordIter = records.records(p.topicPartition).iterator(); @@ -434,7 +405,6 @@ private void readNextBatch() { }); curBatch = Iterators.concat(partitionStates.iterator()); - curRecord = null; } @Override @@ -466,8 +436,9 @@ public boolean advance() throws IOException { ConsumerRecord rawRecord = pState.record; long consumed = pState.consumedOffset; + long offset = rawRecord.offset(); - if (consumed >= 0 && rawRecord.offset() <= consumed) { + if (consumed >= 0 && offset <= consumed) { // this can happen when compression is enabled in Kafka // should we check if the offset is way off from consumedOffset (say 1M more or less) LOG.info("ignoring already consumed offset {} for {}", @@ -477,6 +448,11 @@ public boolean advance() throws IOException { continue; } else { + // sanity check + if (consumed >= 0 && (offset - consumed) != 1) { + LOG.warn("gap in offsets for {} after {}. {} records missing.", + pState.topicPartition, consumed, offset - consumed - 1); + } // apply user decoders curRecord = new KafkaRecord( @@ -502,7 +478,16 @@ public boolean advance() throws IOException { @Override public Instant getWatermark() { - return source.timestampFn.apply(curRecord); + //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); + if (curRecord == null) { + LOG.warn("Why is curRecord null? curTimestamp : {}, numPartitions {} : maxOffset : {}", + curTimestamp, partitionStates.size(), + partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); + return Instant.now().minus(Duration.standardMinutes(2)); + } else { + return curTimestamp.minus(Duration.standardMinutes(2)); + } + } @Override diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 2de9bd3e54..63064683b9 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -3,6 +3,7 @@ import java.util.List; import org.joda.time.Duration; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -15,10 +16,12 @@ import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory; +import com.google.cloud.dataflow.sdk.options.Validation.Required; import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.FlatMapElements; import com.google.cloud.dataflow.sdk.transforms.ParDo; +import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Top; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -46,20 +49,27 @@ public static interface Options extends PipelineOptions { void setSlidingWindowPeriod(Integer value); @Description("Bootstarp Server(s) for Kafka") + @Required String getBootstrapServers(); void setBootstrapServers(String servers); - @Description("Num of Top hashtags") + @Description("One or more topics to read from") + @Required + List getTopics(); + void setTopics(List topics); + + @Description("Number of Top Hashtags") @Default.Integer(10) Integer getNumTopHashtags(); void setNumTopHashtags(Integer count); } + private static final ObjectMapper jsonMapper = new ObjectMapper(); + /** * Emit each of the hashtags in tweet json */ static class ExtractHashtagsFn extends DoFn { - private final ObjectMapper jsonMapper = new ObjectMapper(); @Override public void processElement(ProcessContext ctx) throws Exception { @@ -71,6 +81,19 @@ public void processElement(ProcessContext ctx) throws Exception { } } + // return timestamp from "timestamp_ms" field. + static SerializableFunction timestampFn = json -> { + try { + long timestamp_ms = jsonMapper + .readTree(json) + .path("timestamp_ms") + .asLong(); + return timestamp_ms == 0 ? Instant.now() : new Instant(timestamp_ms); + } catch (Exception e) { + throw new RuntimeException("Incorrect json", e); + } + }; + public static void main(String args[]) { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); @@ -83,20 +106,18 @@ public static void main(String args[]) { UnboundedSource kafkaSource = KafkaSource .unboundedValueSourceBuilder() .withBootstrapServers(options.getBootstrapServers()) - .withTopics(ImmutableList.of("sample_tweets_json")) - //.withConsumerProperty("auto.offset.reset", "earliest") // XXX Temp + .withTopics(options.getTopics()) .withValueDecoderFn(bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) + .withTimestampFn(timestampFn) .build(); pipeline - .apply(Read.from(kafkaSource) - //.withMaxNumRecords(1000)// needed for local runner - .named("sample_tweets")) + .apply(Read.from(kafkaSource).named("sample_tweets")) .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(windowSize)) .every(Duration.standardSeconds(windowPeriod)))) - .apply(Count.perElement()) + .apply(Count.perElement()) .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()).withoutDefaults()) .apply(FlatMapElements .via((List> top) -> { From 2e494ec02626c462a70cee40a5012d3a1ecf2e4c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 09:56:14 -0800 Subject: [PATCH 20/72] more getWatermark() debugging --- .../cloud/dataflow/contrib/kafka/KafkaSource.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 743ae69a19..a5afe80441 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -479,15 +479,15 @@ public boolean advance() throws IOException { @Override public Instant getWatermark() { //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); + LOG.warn("curRec is {}? curTimestamp : {}, numPartitions {} : maxOffset : {}", + (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), + partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); + if (curRecord == null) { - LOG.warn("Why is curRecord null? curTimestamp : {}, numPartitions {} : maxOffset : {}", - curTimestamp, partitionStates.size(), - partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); - return Instant.now().minus(Duration.standardMinutes(2)); + return null; } else { return curTimestamp.minus(Duration.standardMinutes(2)); } - } @Override From ef2620d84686d984a5cdbf3442f54533a1abad74 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 10:22:36 -0800 Subject: [PATCH 21/72] minor --- .../google/cloud/dataflow/contrib/kafka/KafkaSource.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index a5afe80441..c2972b917c 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -479,15 +479,12 @@ public boolean advance() throws IOException { @Override public Instant getWatermark() { //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); - LOG.warn("curRec is {}? curTimestamp : {}, numPartitions {} : maxOffset : {}", + LOG.warn("curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); - if (curRecord == null) { - return null; - } else { - return curTimestamp.minus(Duration.standardMinutes(2)); - } + Instant timestamp = curRecord == null ? Instant.now() : curTimestamp; + return timestamp.minus(Duration.standardMinutes(2)); } @Override From 5e0a48543ff5b601d1657273dd00ba71a211bf12 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 11:21:20 -0800 Subject: [PATCH 22/72] minor --- .../google/cloud/dataflow/contrib/kafka/KafkaSource.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index c2972b917c..9073c8acab 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -264,20 +264,21 @@ public List> generateInitialSplits( int numSplits = Math.min(desiredNumSplits, partitions.size()); + // Map of : split index -> List of indices of partitions assigned to it Map> assignments = IntStream.range(0, partitions.size()) .mapToObj(i -> i) .collect(Collectors.groupingBy(i -> i % numSplits)); // groupingBy preserves order. // create a new source for each split with the assigned partitions for the split return IntStream.range(0, numSplits) - .mapToObj(split -> { - List assignedToSplit = assignments.get(split) + .mapToObj(splitIdx -> { + List assignedToSplit = assignments.get(splitIdx) .stream() .map(i -> partitions.get(i)) .collect(Collectors.toList()); LOG.info("Partitions assigned for split {} : {}", - split, Joiner.on(",").join(assignedToSplit)); + splitIdx, Joiner.on(",").join(assignedToSplit)); // copy of 'this' with assignedPartitions replaced with assignedToSplit return new UnboundedKafkaSource( @@ -479,7 +480,7 @@ public boolean advance() throws IOException { @Override public Instant getWatermark() { //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); - LOG.warn("curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", + LOG.info("curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); From a0ee9e43bc76f1970b5f1eabaed9c4f496462867 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 12:37:57 -0800 Subject: [PATCH 23/72] KafkaRecord --- .../dataflow/contrib/kafka/KafkaRecord.java | 50 +++++++++++++++++++ 1 file changed, 50 insertions(+) create mode 100644 contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java new file mode 100644 index 0000000000..3742de8e11 --- /dev/null +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -0,0 +1,50 @@ +package com.google.cloud.dataflow.contrib.kafka; + +import java.io.Serializable; + +/** + * TODO(rangadi) + * @author rangadi + */ +public class KafkaRecord implements Serializable { + + private final String topic; + private final int partition; + private final long offset; + private final K key; + private final V value; + + public KafkaRecord( + String topic, + int partition, + long offset, + K key, + V value) { + + this.topic = topic; + this.partition = partition; + this.offset = offset; + this.key = key; + this.value = value; + } + + public String getTopic() { + return topic; + } + + public int getPartition() { + return partition; + } + + public long getOffset() { + return offset; + } + + public K getKey() { + return key; + } + + public V getValue() { + return value; + } +} From 25b761c72f9cd74a668356a25727071b9d90d191 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 17:11:37 -0800 Subject: [PATCH 24/72] big update to app --- .../contrib/kafka/KafkaCheckpointMark.java | 13 +- .../dataflow/contrib/kafka/KafkaRecord.java | 3 +- .../dataflow/contrib/kafka/KafkaSource.java | 18 +- .../contrib/kafka/TopHashtagsExample.java | 181 ++++++++++++++---- 4 files changed, 157 insertions(+), 58 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java index e57c941493..f945a7e5cd 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java @@ -13,8 +13,6 @@ /** * Checkpoint for an unbounded KafkaSource reader. Consists of Kafka topic name, partition id, * and the latest offset consumed so far. - * - * @author rangadi */ @DefaultCoder(SerializableCoder.class) public class KafkaCheckpointMark implements UnboundedSource.CheckpointMark, Serializable { @@ -33,12 +31,12 @@ public List getPartitions() { public void finalizeCheckpoint() throws IOException { /* * nothing to do. + * * we might want to support committing offset in Kafka, though it does not guarantee - * no-duplicates, it could support Dataflow restart better. - * Unlike an update of a dataflow job, a restart does not have checkpoint state. - * This secondary checkpoint might be a good start for readers. - * Another similar benefit is when the number of workers or number of Kafka partitions - * changes. + * no-duplicates, it could support Dataflow restart better. Unlike an update of a dataflow job, + * a restart does not have checkpoint state. This secondary checkpoint might be a good start + * for readers. Another similar benefit is when the number of workers or number of + * Kafka partitions changes. */ } @@ -59,6 +57,5 @@ public long getOffset() { return offset; } } - } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index 3742de8e11..634e1aafcb 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -3,8 +3,7 @@ import java.io.Serializable; /** - * TODO(rangadi) - * @author rangadi + * TODO(rangadi): JavaDoc */ public class KafkaRecord implements Serializable { diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 9073c8acab..f31b26a19a 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -57,9 +57,7 @@ import com.google.common.io.Closeables; /** - * TODO(rangadi) - * - * @author rangadi + * TODO(rangadi): JavaDoc */ public class KafkaSource { @@ -335,6 +333,7 @@ private static class PartitionState implements Iterator { private ConsumerRecord record = null; private long consumedOffset; + // might need to keep track of per partition watermark. not decided yet about the semantics PartitionState(TopicPartition partition, long offset) { this.topicPartition = partition; @@ -405,6 +404,7 @@ private void readNextBatch() { p.record = null; }); + // TODO : should we round-robin between different partitions... curBatch = Iterators.concat(partitionStates.iterator()); } @@ -424,9 +424,7 @@ public boolean start() throws IOException { } }); - readNextBatch(); - - return curBatch.hasNext(); + return advance(); } @Override @@ -455,12 +453,13 @@ public boolean advance() throws IOException { pState.topicPartition, consumed, offset - consumed - 1); } + // Ben Chambers... XXX Remove // apply user decoders curRecord = new KafkaRecord( rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), - source.keyDecoderFn.apply(rawRecord.key()), + source.keyDecoderFn.apply(rawRecord.key()), // TODO: use coders rather than functions. source.valueDecoderFn.apply(rawRecord.value())); curTimestamp = source.timestampFn.apply(curRecord); @@ -479,6 +478,11 @@ public boolean advance() throws IOException { @Override public Instant getWatermark() { + // TODO : keep track of per-partition watermark + // user provides watermark fn per partition. + // return min of all the timestamps. what if some topics don't have any data? + // for now we will let users handle this, we can return to it + //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); LOG.info("curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 63064683b9..17cd0453b9 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -1,14 +1,22 @@ package com.google.cloud.dataflow.contrib.kafka; import java.util.List; +import java.util.Map; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; import org.joda.time.Duration; import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.UnboundedSource; @@ -19,19 +27,23 @@ import com.google.cloud.dataflow.sdk.options.Validation.Required; import com.google.cloud.dataflow.sdk.transforms.Count; import com.google.cloud.dataflow.sdk.transforms.DoFn; -import com.google.cloud.dataflow.sdk.transforms.FlatMapElements; import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Top; +import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; -import com.google.cloud.dataflow.sdk.values.TypeDescriptor; import com.google.common.base.Charsets; -import com.google.common.collect.ImmutableList; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; /** - * Every minute, print top English hashtags over 10 last 10 minutes + * This Dataflow app show cases {@link KafkaSource}. The application reads from a Kafka topic + * containing JSON Tweets, calculates top + * hashtags in 10 minute window. The results are written back to a Kafka topic. + * * TODO: Move this out this directory. */ public class TopHashtagsExample { @@ -43,7 +55,7 @@ public static interface Options extends PipelineOptions { Integer getSlidingWindowSize(); void setSlidingWindowSize(Integer value); - @Description("Trigger window size, in minutes") + @Description("Trigger window period, in minutes") @Default.Integer(1) Integer getSlidingWindowPeriod(); void setSlidingWindowPeriod(Integer value); @@ -58,42 +70,17 @@ public static interface Options extends PipelineOptions { List getTopics(); void setTopics(List topics); - @Description("Number of Top Hashtags") + @Description("Number of Top Hashtags to track") @Default.Integer(10) Integer getNumTopHashtags(); void setNumTopHashtags(Integer count); - } - - private static final ObjectMapper jsonMapper = new ObjectMapper(); - /** - * Emit each of the hashtags in tweet json - */ - static class ExtractHashtagsFn extends DoFn { - - @Override - public void processElement(ProcessContext ctx) throws Exception { - for (JsonNode hashtag : jsonMapper.readTree(ctx.element()) - .with("entities") - .withArray("hashtags")) { - ctx.output(hashtag.get("text").asText()); - } - } + @Description("Kafka topic name for writing results") + @Required + String getOutputTopic(); + void setOutputTopic(String topic); } - // return timestamp from "timestamp_ms" field. - static SerializableFunction timestampFn = json -> { - try { - long timestamp_ms = jsonMapper - .readTree(json) - .path("timestamp_ms") - .asLong(); - return timestamp_ms == 0 ? Instant.now() : new Instant(timestamp_ms); - } catch (Exception e) { - throw new RuntimeException("Incorrect json", e); - } - }; - public static void main(String args[]) { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); @@ -116,15 +103,127 @@ public static void main(String args[]) { .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(windowSize)) - .every(Duration.standardSeconds(windowPeriod)))) + .every(Duration.standardMinutes(windowPeriod)))) .apply(Count.perElement()) .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()).withoutDefaults()) - .apply(FlatMapElements - .via((List> top) -> { - LOG.info("Top Hashtags in {} minutes : {}", windowSize, top); - return ImmutableList.of();}) - .withOutputType(new TypeDescriptor(){})); + .apply(ParDo.of(new OutputFormatter())) + .apply(ParDo.of(new KafkaWriter(options))); pipeline.run(); } + + // The rest of the file implements DoFns to extract hashtags, formatting output, writing output + // back to Kafka. Note that writing to Kafka is not a complete Dataflow Sink. It is a best-effort + // logging the results. + + private static final ObjectMapper jsonMapper = new ObjectMapper(); + + /** + * Emit hashtags in the tweet (if any) + */ + private static class ExtractHashtagsFn extends DoFn { + + @Override + public void processElement(ProcessContext ctx) throws Exception { + for (JsonNode hashtag : jsonMapper.readTree(ctx.element()) + .with("entities") + .withArray("hashtags")) { + ctx.output(hashtag.get("text").asText()); + } + } + } + + // extract timestamp from "timestamp_ms" field. + private static SerializableFunction timestampFn = + new SerializableFunction() { + @Override + public Instant apply(String json) { + try { + long timestamp_ms = jsonMapper.readTree(json).path("timestamp_ms").asLong(); + return timestamp_ms == 0 ? Instant.now() : new Instant(timestamp_ms); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + }; + + // return json string containing top hashtags and window information time + private static class OutputFormatter extends DoFn>, String> + implements DoFn.RequiresWindowAccess { + + transient private DateTimeFormatter formatter; + transient private ObjectWriter jsonWriter; + + static class OutputJson { + @JsonProperty String windowStart; + @JsonProperty String windowEnd; + @JsonProperty String generatedAt; + @JsonProperty List topHashtags; + + OutputJson(String windowStart, String windowEnd, + String generatedAt, List topHashtags) { + this.windowStart = windowStart; + this.windowEnd = windowEnd; + this.generatedAt = generatedAt; + this.topHashtags = topHashtags; + } + } + + static class HashtagInfo { + @JsonProperty final String hashtag; + @JsonProperty final long count; + HashtagInfo(String hashtag, long count) { + this.hashtag = hashtag; + this.count = count; + } + } + + @Override + public void processElement(ProcessContext ctx) throws Exception { + if (formatter == null) { + formatter = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").withZoneUTC(); + jsonWriter = new ObjectMapper().writerWithType(OutputJson.class); + } + + List topHashtags = Lists.newArrayListWithCapacity(ctx.element().size()); + + for (KV tag : ctx.element()) { + topHashtags.add(new HashtagInfo(tag.getKey(), tag.getValue())); + } + + IntervalWindow window = (IntervalWindow) ctx.window(); + + String json = jsonWriter.writeValueAsString(new OutputJson( + formatter.print(window.start()), + formatter.print(window.end()), + formatter.print(Instant.now()), + topHashtags)); + + ctx.output(json); + } + } + + private static class KafkaWriter extends DoFn { + + private final String topic; + private final Map config; + private transient KafkaProducer producer = null; + + public KafkaWriter(Options options) { + this.topic = options.getOutputTopic(); + this.config = ImmutableMap.of( + "bootstrap.servers", options.getBootstrapServers(), + "key.serializer", StringSerializer.class.getName(), + "value.serializer", StringSerializer.class.getName()); + } + + @Override + public void processElement(ProcessContext ctx) throws Exception { + if (producer == null) { + producer = new KafkaProducer(config); + } + LOG.info("Top Hashtags : " + ctx.element()); + producer.send(new ProducerRecord<>(topic, ctx.element())); + } + } } From cab32fcac2959324f1dad811930c98342c8dc7aa Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 17:21:51 -0800 Subject: [PATCH 25/72] minor --- .../dataflow/contrib/kafka/TopHashtagsExample.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 17cd0453b9..43584bbf65 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -44,9 +44,22 @@ * containing JSON Tweets, calculates top * hashtags in 10 minute window. The results are written back to a Kafka topic. * + *
+ * Usage:
+ *   $ java -cp jar_with_dependencies.jar                                   \
+ *          com.google.cloud.dataflow.contrib.kafka.TopHashtagsExample      \
+ *          --project=GCP_PROJECT                                           \
+ *          --stagingLocation=GS_STAGING_DIRECTORY                          \
+ *          --runner=BlockingDataflowPipelineRunner                         \
+ *          --bootstrapServers="a_kafka_server:9092"                        \
+ *          --topics="sample_tweets_json"                                   \
+ *          --outputTopic="top_hashtags"
+ * 
+ * * TODO: Move this out this directory. */ public class TopHashtagsExample { + private static final Logger LOG = LoggerFactory.getLogger(TopHashtagsExample.class); public static interface Options extends PipelineOptions { From bf41bec3d81b87a833d21b9a4a668e6454614957 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 25 Feb 2016 23:18:48 -0800 Subject: [PATCH 26/72] use coders rather than functions to encode kafka bytes to user types --- .../dataflow/contrib/kafka/KafkaRecord.java | 15 ++ .../contrib/kafka/KafkaRecordCoder.java | 102 ++++++++++ .../dataflow/contrib/kafka/KafkaSource.java | 178 +++++++++--------- .../contrib/kafka/TopHashtagsExample.java | 30 +-- 4 files changed, 222 insertions(+), 103 deletions(-) create mode 100644 contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index 634e1aafcb..67ee2bc4b3 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -46,4 +46,19 @@ public K getKey() { public V getValue() { return value; } + + @Override + public boolean equals(Object obj) { + if (obj instanceof KafkaRecord) { + @SuppressWarnings("unchecked") + KafkaRecord other = (KafkaRecord) obj; + return topic.equals(other.topic) + && partition == other.partition + && offset == other.offset + && key.equals(other.key) + && value.equals(other.value); + } else { + return false; + } + } } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java new file mode 100644 index 0000000000..34e27f032a --- /dev/null +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java @@ -0,0 +1,102 @@ +package com.google.cloud.dataflow.contrib.kafka; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.CoderException; +import com.google.cloud.dataflow.sdk.coders.KvCoder; +import com.google.cloud.dataflow.sdk.coders.StandardCoder; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; +import com.google.cloud.dataflow.sdk.coders.VarIntCoder; +import com.google.cloud.dataflow.sdk.coders.VarLongCoder; +import com.google.cloud.dataflow.sdk.util.PropertyNames; +import com.google.common.collect.ImmutableList; + +public class KafkaRecordCoder extends StandardCoder> { + + private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); + private static final VarLongCoder longCoder = VarLongCoder.of(); + private static final VarIntCoder intCoder = VarIntCoder.of(); + + private final Coder keyCoder; + private final Coder valueCoder; + + @JsonCreator + public static KafkaRecordCoder of(@JsonProperty(PropertyNames.COMPONENT_ENCODINGS) + List> components) { + KvCoder kvCoder = KvCoder.of(components); + return of(kvCoder.getKeyCoder(), kvCoder.getValueCoder()); + } + + public static KafkaRecordCoder of(Coder keyCoder, Coder valueCoder) { + return new KafkaRecordCoder(keyCoder, valueCoder); + } + + public KafkaRecordCoder(Coder keyCoder, Coder valueCoder) { + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + } + + @Override + public void encode(KafkaRecord value, OutputStream outStream, Context context) + throws CoderException, IOException { + Context nested = context.nested(); + stringCoder.encode(value.getTopic(), outStream, nested); + intCoder .encode(value.getPartition(), outStream, nested); + longCoder .encode(value.getOffset(), outStream, nested); + keyCoder .encode(value.getKey(), outStream, nested); + valueCoder .encode(value.getValue(), outStream, nested); + } + + @Override + public KafkaRecord decode(InputStream inStream, Context context) + throws CoderException, IOException { + Context nested = context.nested(); + return new KafkaRecord( + stringCoder .decode(inStream, nested), + intCoder .decode(inStream, nested), + longCoder .decode(inStream, nested), + keyCoder .decode(inStream, nested), + valueCoder .decode(inStream, nested)); + } + + @Override + public List> getCoderArguments() { + return ImmutableList.of(keyCoder, valueCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + verifyDeterministic("Key and Value coder should be deterministic", keyCoder, valueCoder); + } + + @Override + public boolean isRegisterByteSizeObserverCheap(KafkaRecord value, Context context) { + return keyCoder.isRegisterByteSizeObserverCheap(value.getKey(), context.nested()) + && valueCoder.isRegisterByteSizeObserverCheap(value.getValue(), context.nested()); + //XXX don't we have to implement getEncodedSize()? + } + + @Override + public Object structuralValue(KafkaRecord value) throws Exception{ + if (consistentWithEquals()) + return value; + else + return new KafkaRecord( + value.getTopic(), + value.getPartition(), + value.getOffset(), + keyCoder.structuralValue(value.getKey()), + valueCoder.structuralValue(value.getValue())); + } + + @Override + public boolean consistentWithEquals() { + return keyCoder.consistentWithEquals() && valueCoder.consistentWithEquals(); + } +} diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index f31b26a19a..7ca89c5202 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -40,6 +40,7 @@ import org.slf4j.LoggerFactory; import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; @@ -47,7 +48,7 @@ import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; -import com.google.cloud.dataflow.sdk.values.TypeDescriptor; +import com.google.cloud.dataflow.sdk.util.ExposedByteArrayInputStream; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; @@ -68,7 +69,12 @@ public class KafkaSource { * - confirm non-blocking behavior in advance() */ - private static SerializableFunction identityFn = bytes -> bytes; + private static class NowTimestampFn implements SerializableFunction { + @Override + public Instant apply(T input) { + return Instant.now(); + } + } public static Builder unboundedSourceBuilder() { return new Builder(); @@ -76,8 +82,8 @@ public static Builder unboundedSourceBuilder() { public static Builder unboundedByteSourceBuilder() { return new Builder() - .withKeyDecoderFn(identityFn) - .withValueDecoderFn(identityFn); + .withKeyCoder(ByteArrayCoder.of()) + .withValueCoder(ByteArrayCoder.of()); } /** @@ -87,16 +93,16 @@ public static Builder unboundedByteSourceBuilder() { public static ValueSourceBuilder unboundedValueSourceBuilder() { return new ValueSourceBuilder( new Builder() - .withKeyDecoderFn(identityFn)); + .withKeyCoder(ByteArrayCoder.of())); } public static class Builder { private List topics; // future: let users specify subset of partitions to read - private SerializableFunction keyDecoderFn; - private SerializableFunction valueDecoderFn; - private SerializableFunction, Instant> timestampFn = input -> Instant.now(); + private Coder keyCoder; + private Coder valueCoder; + private SerializableFunction, Instant> timestampFn = new NowTimestampFn<>(); private Map mutableConsumerConfig = Maps.newHashMap(); @@ -157,15 +163,13 @@ public Builder withConsumerProperties(Map configToUpdate) return this; } - public Builder withKeyDecoderFn( - SerializableFunction keyDecoderFn) { - this.keyDecoderFn = keyDecoderFn; + public Builder withKeyCoder(Coder keyCoder) { + this.keyCoder = keyCoder; return this; } - public Builder withValueDecoderFn( - SerializableFunction valueDecoderFn) { - this.valueDecoderFn = valueDecoderFn; + public Builder withValueCoder(Coder valueCoder) { + this.valueCoder = valueCoder; return this; } @@ -188,14 +192,14 @@ public UnboundedSource, KafkaCheckpointMark> build() { "Kafka bootstrap servers should be set"); Preconditions.checkNotNull(topics, "Kafka topics should be set"); Preconditions.checkArgument(!topics.isEmpty(), "At least one topic is required"); - Preconditions.checkNotNull(keyDecoderFn, "Decoder for Kafka key bytes should be set"); - Preconditions.checkNotNull(valueDecoderFn, "Decoder for Kafka values bytes should be set"); + Preconditions.checkNotNull(keyCoder, "Coder for Kafka key bytes is required"); + Preconditions.checkNotNull(valueCoder, "Coder for Kafka values bytes is required"); return new UnboundedKafkaSource( consumerConfig, topics, - keyDecoderFn, - valueDecoderFn, + keyCoder, + valueCoder, timestampFn, ImmutableList.of() // no assigned partitions yet ); @@ -210,23 +214,23 @@ private static class UnboundedKafkaSource private final ImmutableMap consumerConfig; private final List topics; - private final SerializableFunction keyDecoderFn; - private final SerializableFunction valueDecoderFn; + private final Coder keyCoder; + private final Coder valueCoder; private final SerializableFunction, Instant> timestampFn; private final List assignedPartitions; public UnboundedKafkaSource( ImmutableMap consumerConfig, List topics, - SerializableFunction keyDecoderFn, - SerializableFunction valueDecoderFn, + Coder keyCoder, + Coder valueCoder, SerializableFunction, Instant> timestampFn, List assignedPartitions) { this.consumerConfig = consumerConfig; this.topics = topics; - this.keyDecoderFn = keyDecoderFn; - this.valueDecoderFn = valueDecoderFn; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; this.timestampFn = timestampFn; this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); } @@ -282,8 +286,8 @@ public List> generateInitialSplits( return new UnboundedKafkaSource( this.consumerConfig, this.topics, - this.keyDecoderFn, - this.valueDecoderFn, + this.keyCoder, + this.valueCoder, this.timestampFn, assignedToSplit); }) @@ -313,9 +317,7 @@ public void validate() { @Override public Coder> getDefaultOutputCoder() { - // no coder is logically needed. user explicitly provides functions to decode key and value - // sdk requires a Coder. - return SerializableCoder.of(new TypeDescriptor>() {}); + return KafkaRecordCoder.of(keyCoder, valueCoder); } } @@ -370,7 +372,7 @@ public UnboundedKafkaReader( .map(tp -> new PartitionState(tp, -1L)) .iterator()); - // a) verify that assigned and check-pointed partitions match + // a) verify that assigned and check-pointed partitions match exactly // b) set consumed offsets if (checkpointMark != null) { @@ -440,33 +442,28 @@ public boolean advance() throws IOException { if (consumed >= 0 && offset <= consumed) { // this can happen when compression is enabled in Kafka // should we check if the offset is way off from consumedOffset (say 1M more or less) - LOG.info("ignoring already consumed offset {} for {}", - rawRecord.offset(), pState.topicPartition); - - // TODO: increment a counter? + LOG.info("ignoring already consumed offset {} for {}", offset, pState.topicPartition); continue; + } - } else { - // sanity check - if (consumed >= 0 && (offset - consumed) != 1) { - LOG.warn("gap in offsets for {} after {}. {} records missing.", - pState.topicPartition, consumed, offset - consumed - 1); - } - - // Ben Chambers... XXX Remove - // apply user decoders - curRecord = new KafkaRecord( - rawRecord.topic(), - rawRecord.partition(), - rawRecord.offset(), - source.keyDecoderFn.apply(rawRecord.key()), // TODO: use coders rather than functions. - source.valueDecoderFn.apply(rawRecord.value())); - - curTimestamp = source.timestampFn.apply(curRecord); - pState.consumedOffset = rawRecord.offset(); - - return true; + // sanity check + if (consumed >= 0 && (offset - consumed) != 1) { + LOG.warn("gap in offsets for {} after {}. {} records missing.", + pState.topicPartition, consumed, offset - consumed - 1); } + + // decode with user coders + curRecord = new KafkaRecord( + rawRecord.topic(), + rawRecord.partition(), + rawRecord.offset(), + decode(rawRecord.key(), source.keyCoder), + decode(rawRecord.value(), source.valueCoder)); + + curTimestamp = source.timestampFn.apply(curRecord); + pState.consumedOffset = rawRecord.offset(); + + return true; } else { readNextBatch(); @@ -476,6 +473,12 @@ public boolean advance() throws IOException { } } + private static T decode(byte[] bytes, Coder coder) throws IOException { + if (bytes == null) + return null; + return coder.decode(new ExposedByteArrayInputStream(bytes), Coder.Context.OUTER); + } + @Override public Instant getWatermark() { // TODO : keep track of per-partition watermark @@ -524,42 +527,41 @@ public void close() throws IOException { // Builder, Source, Reader wrappers when user is only interested in Value in KafkaRecord : - public static class ValueSourceBuilder { - // TODO : remove 'extends Serializable' restriction or improve it to just require a Coder + public static class ValueSourceBuilder { - private Builder underlying; + private Builder underlying; - private ValueSourceBuilder(Builder underlying) { + private ValueSourceBuilder(Builder underlying) { this.underlying = underlying; } - public ValueSourceBuilder withBootstrapServers(String bootstrapServers) { - return new ValueSourceBuilder(underlying.withBootstrapServers(bootstrapServers)); + public ValueSourceBuilder withBootstrapServers(String bootstrapServers) { + return new ValueSourceBuilder(underlying.withBootstrapServers(bootstrapServers)); } - public ValueSourceBuilder withTopics(Collection topics) { - return new ValueSourceBuilder(underlying.withTopics(topics)); + public ValueSourceBuilder withTopics(Collection topics) { + return new ValueSourceBuilder(underlying.withTopics(topics)); } - public ValueSourceBuilder withConsumerProperty(String configKey, Object configValue) { - return new ValueSourceBuilder(underlying.withConsumerProperty(configKey, configValue)); + public ValueSourceBuilder withConsumerProperty(String configKey, Object configValue) { + return new ValueSourceBuilder(underlying.withConsumerProperty(configKey, configValue)); } - public ValueSourceBuilder withConsumerProperties(Map configToUpdate) { - return new ValueSourceBuilder(underlying.withConsumerProperties(configToUpdate)); + public ValueSourceBuilder withConsumerProperties(Map configToUpdate) { + return new ValueSourceBuilder(underlying.withConsumerProperties(configToUpdate)); } - public ValueSourceBuilder withValueDecoderFn(SerializableFunction valueDecoderFn) { - return new ValueSourceBuilder(underlying.withValueDecoderFn(valueDecoderFn)); + public ValueSourceBuilder withValueCoder(Coder valueCoder) { + return new ValueSourceBuilder(underlying.withValueCoder(valueCoder)); } - public ValueSourceBuilder withTimestampFn(SerializableFunction timestampFn) { - return new ValueSourceBuilder( + public ValueSourceBuilder withTimestampFn(SerializableFunction timestampFn) { + return new ValueSourceBuilder( underlying.withTimestampFn(record -> timestampFn.apply(record.getValue()))); } - public UnboundedSource build() { - return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); + public UnboundedSource build() { + return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); } } @@ -567,27 +569,27 @@ public UnboundedSource build() { * Usually the users are only interested in value in KafkaRecord. This is a convenient class * to strip out other fields in KafkaRecord returned by UnboundedKafkaValueSource */ - private static class UnboundedKafkaValueSource extends UnboundedSource { + private static class UnboundedKafkaValueSource extends UnboundedSource { - private final UnboundedKafkaSource underlying; + private final UnboundedKafkaSource underlying; - public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { + public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { this.underlying = underlying; } @Override - public List> generateInitialSplits( + public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { return underlying .generateInitialSplits(desiredNumSplits, options) .stream() - .map(s -> new UnboundedKafkaValueSource(s)) + .map(s -> new UnboundedKafkaValueSource(s)) .collect(Collectors.toList()); } @Override - public UnboundedReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { - return new UnboundedKafkaValueReader(this, underlying.createReader(options, checkpointMark)); + public UnboundedReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { + return new UnboundedKafkaValueReader(this, underlying.createReader(options, checkpointMark)); } @Override @@ -601,18 +603,18 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { - return SerializableCoder.of(new TypeDescriptor() {}); + public Coder getDefaultOutputCoder() { + return underlying.valueCoder; } } - private static class UnboundedKafkaValueReader extends UnboundedReader { + private static class UnboundedKafkaValueReader extends UnboundedReader { - private final UnboundedKafkaValueSource source; - private final UnboundedKafkaReader underlying; + private final UnboundedKafkaValueSource source; + private final UnboundedKafkaReader underlying; - public UnboundedKafkaValueReader(UnboundedKafkaValueSource source, - UnboundedKafkaReader underlying) { + public UnboundedKafkaValueReader(UnboundedKafkaValueSource source, + UnboundedKafkaReader underlying) { this.source = source; this.underlying = underlying; } @@ -638,12 +640,12 @@ public CheckpointMark getCheckpointMark() { } @Override - public UnboundedKafkaValueSource getCurrentSource() { + public UnboundedKafkaValueSource getCurrentSource() { return source; } @Override - public T getCurrent() throws NoSuchElementException { + public V getCurrent() throws NoSuchElementException { return underlying.getCurrent().getValue(); } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index 43584bbf65..e17b3a462a 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; import com.google.cloud.dataflow.sdk.Pipeline; +import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.options.Default; @@ -34,7 +35,6 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; import com.google.cloud.dataflow.sdk.values.KV; -import com.google.common.base.Charsets; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; @@ -107,7 +107,7 @@ public static void main(String args[]) { .unboundedValueSourceBuilder() .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) - .withValueDecoderFn(bytes -> (bytes == null) ? null : new String(bytes, Charsets.UTF_8)) + .withValueCoder(StringUtf8Coder.of()) .withTimestampFn(timestampFn) .build(); @@ -214,21 +214,21 @@ public void processElement(ProcessContext ctx) throws Exception { ctx.output(json); } - } + } - private static class KafkaWriter extends DoFn { + private static class KafkaWriter extends DoFn { - private final String topic; - private final Map config; - private transient KafkaProducer producer = null; + private final String topic; + private final Map config; + private transient KafkaProducer producer = null; - public KafkaWriter(Options options) { - this.topic = options.getOutputTopic(); - this.config = ImmutableMap.of( - "bootstrap.servers", options.getBootstrapServers(), - "key.serializer", StringSerializer.class.getName(), - "value.serializer", StringSerializer.class.getName()); - } + public KafkaWriter(Options options) { + this.topic = options.getOutputTopic(); + this.config = ImmutableMap.of( + "bootstrap.servers", options.getBootstrapServers(), + "key.serializer", StringSerializer.class.getName(), + "value.serializer", StringSerializer.class.getName()); + } @Override public void processElement(ProcessContext ctx) throws Exception { @@ -238,5 +238,5 @@ public void processElement(ProcessContext ctx) throws Exception { LOG.info("Top Hashtags : " + ctx.element()); producer.send(new ProducerRecord<>(topic, ctx.element())); } - } + } } From c02f00213764147aacfc4e1d4f8c2b2721e18097 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 26 Feb 2016 11:35:24 -0800 Subject: [PATCH 27/72] stay with java 1.7 (both for source and target) --- contrib/kafka/pom.xml | 4 +- .../dataflow/contrib/kafka/KafkaSource.java | 253 +++++++++++------- .../contrib/kafka/TopHashtagsExample.java | 6 +- 3 files changed, 165 insertions(+), 98 deletions(-) diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index a5fdc2e30f..15447c953c 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -46,8 +46,8 @@ maven-compiler-plugin 3.2 - 1.8 - 1.8 + 1.7 + 1.7
diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 7ca89c5202..aea24ae410 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -19,12 +19,13 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.NoSuchElementException; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import javax.annotation.Nullable; @@ -32,14 +33,15 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.api.client.util.Maps; +import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; @@ -49,12 +51,15 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.util.ExposedByteArrayInputStream; +import com.google.common.base.Function; import com.google.common.base.Joiner; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; import com.google.common.io.Closeables; /** @@ -103,6 +108,7 @@ public static class Builder { private Coder keyCoder; private Coder valueCoder; private SerializableFunction, Instant> timestampFn = new NowTimestampFn<>(); + private Optional, Instant>> watermarkFn = Optional.absent(); private Map mutableConsumerConfig = Maps.newHashMap(); @@ -154,12 +160,13 @@ public Builder withConsumerProperty(String configKey, Object configValue) } /** - * Update consumer config properties. Note that this does not not discard already configurured. + * Update consumer config properties. Note that this does not not discard already configured. * Same as invoking #withConsumerProperty() with each entry. */ public Builder withConsumerProperties(Map configToUpdate) { - configToUpdate.entrySet().stream().forEach( - e -> withConsumerProperty(e.getKey(), e.getValue())); + for(Entry e : configToUpdate.entrySet()) { + withConsumerProperty(e.getKey(), e.getValue()); + } return this; } @@ -174,8 +181,8 @@ public Builder withValueCoder(Coder valueCoder) { } /** - * Set a timestamp function. Default is the timestamp when the ConsumerRecord is processed - * by {@UnboundedReader#advance()} + * A function to assign a timestamp to a record. Default is the timestamp when the + * record is processed by {@UnboundedReader#advance()} */ public Builder withTimestampFn( SerializableFunction, Instant> timestampFn) { @@ -183,6 +190,15 @@ public Builder withTimestampFn( return this; } + /** + * A function to calculate watermark. When this is not set, last record timestamp is returned + * in {@link UnboundedReader#getWatermark()}. + */ + public Builder withWatermarkFn(SerializableFunction, Instant> watermarkFn) { + this.watermarkFn = Optional.of(watermarkFn); + return this; + } + public UnboundedSource, KafkaCheckpointMark> build() { ImmutableMap consumerConfig = ImmutableMap.copyOf(mutableConsumerConfig); @@ -201,7 +217,8 @@ public UnboundedSource, KafkaCheckpointMark> build() { keyCoder, valueCoder, timestampFn, - ImmutableList.of() // no assigned partitions yet + watermarkFn, + ImmutableList.of() // no assigned partitions yet ); } } @@ -217,6 +234,8 @@ private static class UnboundedKafkaSource private final Coder keyCoder; private final Coder valueCoder; private final SerializableFunction, Instant> timestampFn; + // would it be a good idea to pass currentTimestamp to watermarkFn? + private final Optional, Instant>> watermarkFn; private final List assignedPartitions; public UnboundedKafkaSource( @@ -225,6 +244,7 @@ public UnboundedKafkaSource( Coder keyCoder, Coder valueCoder, SerializableFunction, Instant> timestampFn, + Optional, Instant>> watermarkFn, List assignedPartitions) { this.consumerConfig = consumerConfig; @@ -232,6 +252,7 @@ public UnboundedKafkaSource( this.keyCoder = keyCoder; this.valueCoder = valueCoder; this.timestampFn = timestampFn; + this.watermarkFn = watermarkFn; this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); } @@ -241,57 +262,66 @@ public List> generateInitialSplits( KafkaConsumer consumer = new KafkaConsumer(consumerConfig); - List partitions; + List partitions = Lists.newArrayList(); + + // fetch partitions for each topic + // sort them in order + // round-robin assign the partition to splits try { - // fetch partitions for each topic and sort them in order. - // sort by partitionId so that topics are evenly distributed among the splits. - - partitions = topics - .stream() - .flatMap(topic -> consumer.partitionsFor(topic).stream()) - .map(partInfo -> new TopicPartition(partInfo.topic(), partInfo.partition())) - .sorted((p1, p2) -> ComparisonChain.start() // sort by - .compare(p1.partition(), p2.partition()) - .compare(p1.topic(), p2.topic()) - .result()) - .collect(Collectors.toList()); + for (String topic : topics) { + for (PartitionInfo p : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(p.topic(), p.partition())); + } + } } finally { consumer.close(); } + Collections.sort(partitions, new Comparator() { + public int compare(TopicPartition tp1, TopicPartition tp2) { + return ComparisonChain + .start() + .compare(tp1.topic(), tp2.topic()) + .compare(tp1.partition(), tp2.partition()) + .result(); + } + }); + Preconditions.checkArgument(desiredNumSplits > 0); Preconditions.checkState(partitions.size() > 0, "Could not find any partitions. Please check Kafka configuration and topic names"); int numSplits = Math.min(desiredNumSplits, partitions.size()); - // Map of : split index -> List of indices of partitions assigned to it - Map> assignments = IntStream.range(0, partitions.size()) - .mapToObj(i -> i) - .collect(Collectors.groupingBy(i -> i % numSplits)); // groupingBy preserves order. + List> assignments = Lists.newArrayList(); + + for (int i=0; inewArrayList()); + } + + for (int i=0; i> result = Lists.newArrayList(); - // create a new source for each split with the assigned partitions for the split - return IntStream.range(0, numSplits) - .mapToObj(splitIdx -> { - List assignedToSplit = assignments.get(splitIdx) - .stream() - .map(i -> partitions.get(i)) - .collect(Collectors.toList()); + for (int i=0; i( + result.add( + new UnboundedKafkaSource( this.consumerConfig, this.topics, this.keyCoder, this.valueCoder, this.timestampFn, - assignedToSplit); - }) - .collect(Collectors.toList()); + this.watermarkFn, + assignments.get(i))); + } + + return result; } @Override @@ -328,11 +358,10 @@ private static class UnboundedKafkaReader private KafkaConsumer consumer; // maintains state of each assigned partition - private static class PartitionState implements Iterator { + private static class PartitionState { private final TopicPartition topicPartition; private Iterator> recordIter = Iterators.emptyIterator(); - private ConsumerRecord record = null; private long consumedOffset; // might need to keep track of per partition watermark. not decided yet about the semantics @@ -341,17 +370,6 @@ private static class PartitionState implements Iterator { this.topicPartition = partition; this.consumedOffset = offset; } - - @Override - public boolean hasNext() { - return recordIter.hasNext(); - } - - @Override - public PartitionState next() { - record = recordIter.next(); - return this; - } } private List partitionStates; @@ -367,10 +385,12 @@ public UnboundedKafkaReader( this.source = source; - partitionStates = ImmutableList.copyOf(source.assignedPartitions - .stream() - .map(tp -> new PartitionState(tp, -1L)) - .iterator()); + partitionStates = ImmutableList.copyOf(Lists.transform(source.assignedPartitions, + new Function() { + public PartitionState apply(TopicPartition tp) { + return new PartitionState(tp, -1L); + } + })); // a) verify that assigned and check-pointed partitions match exactly // b) set consumed offsets @@ -395,19 +415,23 @@ public UnboundedKafkaReader( } private void readNextBatch() { - // read one batch of records. one consumer.poll() + // read one batch of records with single consumer.poll() (may not have any records) - ConsumerRecords records = consumer.poll(10); // what should the timeout be? + ConsumerRecords records = consumer.poll(10); - // increment a counter or stat? + List withRecords = Lists.newLinkedList(); - partitionStates.stream().forEach(p -> { - p.recordIter = records.records(p.topicPartition).iterator(); - p.record = null; - }); + for (PartitionState pState : partitionStates) { + List> pRecords = records.records(pState.topicPartition); - // TODO : should we round-robin between different partitions... - curBatch = Iterators.concat(partitionStates.iterator()); + if (pRecords.size() > 0) { + pState.recordIter = pRecords.iterator(); + withRecords.add(pState); + } + }; + + // cycle through these partitions so that we round-robin among them while returning records + curBatch = Iterators.cycle(withRecords); } @Override @@ -417,31 +441,43 @@ public boolean start() throws IOException { consumer.assign(source.assignedPartitions); // seek to next offset if consumedOffset is set - partitionStates.stream().forEach(p -> { + for(PartitionState p : partitionStates) { if (p.consumedOffset >= 0) { LOG.info("Reader: resuming {} at {}", p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset + 1); } else { LOG.info("Reader: resuming from default offset for {}", p.topicPartition); } - }); + } return advance(); } @Override public boolean advance() throws IOException { + /* Read first record (if any). we need to loop here because : + * - (a) some records initially need to be skipped since they are before consumedOffset + * - (b) when the current batch empty, we want to readNextBatch() and then advance. + * - (c) curBatch is an iterator of iterators and we want to interleave the records from each. + * curBatch.next() might return an empty iterator. + */ while (true) { if (curBatch.hasNext()) { PartitionState pState = curBatch.next(); - ConsumerRecord rawRecord = pState.record; + if (!pState.recordIter.hasNext()) { // -- (c) + pState.recordIter = Iterators.emptyIterator(); // drop ref + curBatch.remove(); + continue; + } + + ConsumerRecord rawRecord = pState.recordIter.next(); long consumed = pState.consumedOffset; long offset = rawRecord.offset(); - if (consumed >= 0 && offset <= consumed) { + if (consumed >= 0 && offset <= consumed) { // -- (a) // this can happen when compression is enabled in Kafka - // should we check if the offset is way off from consumedOffset (say 1M more or less) + // should we check if the offset is way off from consumedOffset (say > 1M)? LOG.info("ignoring already consumed offset {} for {}", offset, pState.topicPartition); continue; } @@ -452,7 +488,7 @@ public boolean advance() throws IOException { pState.topicPartition, consumed, offset - consumed - 1); } - // decode with user coders + // apply user coders curRecord = new KafkaRecord( rawRecord.topic(), rawRecord.partition(), @@ -464,7 +500,7 @@ public boolean advance() throws IOException { pState.consumedOffset = rawRecord.offset(); return true; - } else { + } else { // -- (b) readNextBatch(); if (!curBatch.hasNext()) @@ -475,7 +511,7 @@ public boolean advance() throws IOException { private static T decode(byte[] bytes, Coder coder) throws IOException { if (bytes == null) - return null; + return null; // is this the right thing to do? return coder.decode(new ExposedByteArrayInputStream(bytes), Coder.Context.OUTER); } @@ -486,21 +522,34 @@ public Instant getWatermark() { // return min of all the timestamps. what if some topics don't have any data? // for now we will let users handle this, we can return to it - //XXX what should do? why is curRecord is null? return source.timestampFn.apply(curRecord); - LOG.info("curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", + //XXX what should we do? why is curRecord is null? return source.timestampFn.apply(curRecord); + LOG.info("XXX curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), - partitionStates.stream().collect(Collectors.summarizingLong(s -> s.consumedOffset)).getMax()); + Collections.max(partitionStates, new Comparator() { + public int compare(PartitionState p1, PartitionState p2) { + return (int) (p1.consumedOffset - p2.consumedOffset); + } + })); - Instant timestamp = curRecord == null ? Instant.now() : curTimestamp; - return timestamp.minus(Duration.standardMinutes(2)); + if (curRecord == null) // XXX TEMP + return curTimestamp; + + if (source.watermarkFn.isPresent()) + return source.watermarkFn.get().apply(curRecord); + else + return curTimestamp; } @Override public CheckpointMark getCheckpointMark() { - return new KafkaCheckpointMark(partitionStates - .stream() - .map(p -> new KafkaCheckpointMark.PartitionMark(p.topicPartition, p.consumedOffset)) - .collect(Collectors.toList())); + return new KafkaCheckpointMark( + ImmutableList.copyOf(Lists.transform(partitionStates, // avoid lazy (consumedOffset can change) + new Function() { + public PartitionMark apply(PartitionState p) { + return new PartitionMark(p.topicPartition, p.consumedOffset); + } + } + ))); } @Override @@ -516,7 +565,7 @@ public KafkaRecord getCurrent() throws NoSuchElementException { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { - return curTimestamp; //TODO: how is this related to getWatermark(); + return curTimestamp; } @Override @@ -557,12 +606,30 @@ public ValueSourceBuilder withValueCoder(Coder valueCoder) { public ValueSourceBuilder withTimestampFn(SerializableFunction timestampFn) { return new ValueSourceBuilder( - underlying.withTimestampFn(record -> timestampFn.apply(record.getValue()))); + underlying.withTimestampFn(unwrapKafkaAndThen(timestampFn))); + } + + /** + * A function to calculate watermark. When this is not set, last record timestamp is returned + * in {@link UnboundedReader#getWatermark()}. + */ + public ValueSourceBuilder withWatermarkFn(SerializableFunction watermarkFn) { + return new ValueSourceBuilder( + underlying.withTimestampFn(unwrapKafkaAndThen(watermarkFn))); } public UnboundedSource build() { return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); } + + private static + SerializableFunction, O> unwrapKafkaAndThen(final SerializableFunction fn) { + return new SerializableFunction, O>() { + public O apply(KafkaRecord record) { + return fn.apply(record.getValue()); + } + }; + } } /** @@ -571,20 +638,21 @@ public UnboundedSource build() { */ private static class UnboundedKafkaValueSource extends UnboundedSource { - private final UnboundedKafkaSource underlying; + private final UnboundedKafkaSource underlying; - public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { + public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { this.underlying = underlying; } @Override public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { - return underlying - .generateInitialSplits(desiredNumSplits, options) - .stream() - .map(s -> new UnboundedKafkaValueSource(s)) - .collect(Collectors.toList()); + return Lists.transform(underlying.generateInitialSplits(desiredNumSplits, options), + new Function, UnboundedKafkaValueSource>() { + public UnboundedKafkaValueSource apply(UnboundedKafkaSource input) { + return new UnboundedKafkaValueSource(input); + } + }); } @Override @@ -659,5 +727,4 @@ public void close() throws IOException { underlying.close(); } } - } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java index e17b3a462a..e96caf0120 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java @@ -117,7 +117,7 @@ public static void main(String args[]) { .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(windowSize)) .every(Duration.standardMinutes(windowPeriod)))) - .apply(Count.perElement()) + .apply(Count.perElement()) .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()).withoutDefaults()) .apply(ParDo.of(new OutputFormatter())) .apply(ParDo.of(new KafkaWriter(options))); @@ -224,7 +224,7 @@ private static class KafkaWriter extends DoFn { public KafkaWriter(Options options) { this.topic = options.getOutputTopic(); - this.config = ImmutableMap.of( + this.config = ImmutableMap.of( "bootstrap.servers", options.getBootstrapServers(), "key.serializer", StringSerializer.class.getName(), "value.serializer", StringSerializer.class.getName()); @@ -236,7 +236,7 @@ public void processElement(ProcessContext ctx) throws Exception { producer = new KafkaProducer(config); } LOG.info("Top Hashtags : " + ctx.element()); - producer.send(new ProducerRecord<>(topic, ctx.element())); + producer.send(new ProducerRecord(topic, ctx.element())); } } } From 88df84b1effd4ab6261ccbcefc6a654c22490700 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 26 Feb 2016 11:47:56 -0800 Subject: [PATCH 28/72] minor --- .../com/google/cloud/dataflow/contrib/kafka/KafkaSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index aea24ae410..2766fc24f2 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -532,7 +532,7 @@ public int compare(PartitionState p1, PartitionState p2) { })); if (curRecord == null) // XXX TEMP - return curTimestamp; + return Instant.now(); if (source.watermarkFn.isPresent()) return source.watermarkFn.get().apply(curRecord); From c397dfae29b89557f0db29b07abf7dff1d30a55a Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 26 Feb 2016 18:52:16 -0800 Subject: [PATCH 29/72] tweak timeout for cosumer.poll (longer timeout initially). remove debug logging --- .../dataflow/contrib/kafka/KafkaSource.java | 60 ++++++++++--------- 1 file changed, 33 insertions(+), 27 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 2766fc24f2..23914d1850 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -212,6 +212,7 @@ public UnboundedSource, KafkaCheckpointMark> build() { Preconditions.checkNotNull(valueCoder, "Coder for Kafka values bytes is required"); return new UnboundedKafkaSource( + -1, consumerConfig, topics, keyCoder, @@ -229,6 +230,7 @@ private KafkaSource() {} private static class UnboundedKafkaSource extends UnboundedSource, KafkaCheckpointMark> { + private final int id; // split id, mainly for debugging private final ImmutableMap consumerConfig; private final List topics; private final Coder keyCoder; @@ -239,6 +241,7 @@ private static class UnboundedKafkaSource private final List assignedPartitions; public UnboundedKafkaSource( + int id, ImmutableMap consumerConfig, List topics, Coder keyCoder, @@ -247,6 +250,7 @@ public UnboundedKafkaSource( Optional, Instant>> watermarkFn, List assignedPartitions) { + this.id = id; this.consumerConfig = consumerConfig; this.topics = topics; this.keyCoder = keyCoder; @@ -307,18 +311,21 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { List> result = Lists.newArrayList(); for (int i=0; i assignedToSplit = assignments.get(i); - LOG.info("Partitions assigned to split {} : {}", i, Joiner.on(",").join(assignments.get(i))); + LOG.info("Partitions assigned to split {} (total {}): {}", + i, assignedToSplit.size(), Joiner.on(",").join(assignedToSplit)); result.add( new UnboundedKafkaSource( + i, this.consumerConfig, this.topics, this.keyCoder, this.valueCoder, this.timestampFn, this.watermarkFn, - assignments.get(i))); + assignedToSplit)); } return result; @@ -377,6 +384,8 @@ private static class PartitionState { private KafkaRecord curRecord; private Instant curTimestamp; + private long numRecordsRead = 0; + private Iterator curBatch = Iterators.emptyIterator(); public UnboundedKafkaReader( @@ -415,23 +424,27 @@ public PartitionState apply(TopicPartition tp) { } private void readNextBatch() { - // read one batch of records with single consumer.poll() (may not have any records) + // read one batch of records with single consumer.poll() (may not return any records) + + // longer timeout during initialization. Consumer does not seem to handle short polls very + // well during initialization. It takes much longer to read first record with short timeouts. + long timeout_ms = numRecordsRead > 0 ? 100 : 3000; - ConsumerRecords records = consumer.poll(10); + ConsumerRecords records = consumer.poll(timeout_ms); - List withRecords = Lists.newLinkedList(); + List nonEmpty = Lists.newLinkedList(); for (PartitionState pState : partitionStates) { List> pRecords = records.records(pState.topicPartition); if (pRecords.size() > 0) { pState.recordIter = pRecords.iterator(); - withRecords.add(pState); + nonEmpty.add(pState); } }; - // cycle through these partitions so that we round-robin among them while returning records - curBatch = Iterators.cycle(withRecords); + // cycle through the partitions in order to interleave records from each of them. + curBatch = Iterators.cycle(nonEmpty); } @Override @@ -443,10 +456,10 @@ public boolean start() throws IOException { // seek to next offset if consumedOffset is set for(PartitionState p : partitionStates) { if (p.consumedOffset >= 0) { - LOG.info("Reader: resuming {} at {}", p.topicPartition, p.consumedOffset + 1); + LOG.info("Reader {} : resuming {} at {}", source.id, p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset + 1); } else { - LOG.info("Reader: resuming from default offset for {}", p.topicPartition); + LOG.info("Reader {} : resuming {} at default offset", source.id, p.topicPartition); } } @@ -497,14 +510,18 @@ public boolean advance() throws IOException { decode(rawRecord.value(), source.valueCoder)); curTimestamp = source.timestampFn.apply(curRecord); - pState.consumedOffset = rawRecord.offset(); + pState.consumedOffset = offset; + numRecordsRead++; return true; + } else { // -- (b) + readNextBatch(); - if (!curBatch.hasNext()) + if (!curBatch.hasNext()) { return false; + } } } } @@ -517,22 +534,11 @@ private static T decode(byte[] bytes, Coder coder) throws IOException { @Override public Instant getWatermark() { - // TODO : keep track of per-partition watermark - // user provides watermark fn per partition. - // return min of all the timestamps. what if some topics don't have any data? - // for now we will let users handle this, we can return to it - - //XXX what should we do? why is curRecord is null? return source.timestampFn.apply(curRecord); - LOG.info("XXX curRec is {}. curTimestamp : {}, numPartitions {} : maxOffset : {}", - (curRecord == null) ? "null" : "not null", curTimestamp, partitionStates.size(), - Collections.max(partitionStates, new Comparator() { - public int compare(PartitionState p1, PartitionState p2) { - return (int) (p1.consumedOffset - p2.consumedOffset); - } - })); - - if (curRecord == null) // XXX TEMP + if (numRecordsRead == 0) { + // haven't read any record yet. what is the right thing to do here? + // may be better to invoke user provided watermarkFn with null and let user decide? return Instant.now(); + } if (source.watermarkFn.isPresent()) return source.watermarkFn.get().apply(curRecord); From e953c84e34432f5607a93b02121bd51fb881207c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Sat, 27 Feb 2016 11:32:31 -0800 Subject: [PATCH 30/72] tried consumer metadata fetch rather than longer timeout for first fetch. It didn't work. some more logging updates --- .../dataflow/contrib/kafka/KafkaSource.java | 79 ++++++++++--------- 1 file changed, 43 insertions(+), 36 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 23914d1850..6efa934a80 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -269,7 +269,7 @@ public List> generateInitialSplits( List partitions = Lists.newArrayList(); // fetch partitions for each topic - // sort them in order + // sort by // round-robin assign the partition to splits try { @@ -316,16 +316,15 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { LOG.info("Partitions assigned to split {} (total {}): {}", i, assignedToSplit.size(), Joiner.on(",").join(assignedToSplit)); - result.add( - new UnboundedKafkaSource( - i, - this.consumerConfig, - this.topics, - this.keyCoder, - this.valueCoder, - this.timestampFn, - this.watermarkFn, - assignedToSplit)); + result.add(new UnboundedKafkaSource( + i, + this.consumerConfig, + this.topics, + this.keyCoder, + this.valueCoder, + this.timestampFn, + this.watermarkFn, + assignedToSplit)); } return result; @@ -362,6 +361,7 @@ private static class UnboundedKafkaReader extends UnboundedReader> { private final UnboundedKafkaSource source; + private final String name; private KafkaConsumer consumer; // maintains state of each assigned partition @@ -384,15 +384,18 @@ private static class PartitionState { private KafkaRecord curRecord; private Instant curTimestamp; - private long numRecordsRead = 0; - private Iterator curBatch = Iterators.emptyIterator(); + public String toString() { + return name; + } + public UnboundedKafkaReader( UnboundedKafkaSource source, @Nullable KafkaCheckpointMark checkpointMark) { this.source = source; + this.name = "Reader-" + source.id; partitionStates = ImmutableList.copyOf(Lists.transform(source.assignedPartitions, new Function() { @@ -405,6 +408,8 @@ public PartitionState apply(TopicPartition tp) { // b) set consumed offsets if (checkpointMark != null) { + // set consumed offset + Preconditions.checkState( checkpointMark.getPartitions().size() == source.assignedPartitions.size(), "checkPointMark and assignedPartitions should match"); @@ -423,46 +428,45 @@ public PartitionState apply(TopicPartition tp) { } } - private void readNextBatch() { + private void readNextBatch(boolean isFirstFetch) { // read one batch of records with single consumer.poll() (may not return any records) - // longer timeout during initialization. Consumer does not seem to handle short polls very - // well during initialization. It takes much longer to read first record with short timeouts. - long timeout_ms = numRecordsRead > 0 ? 100 : 3000; - + // Use a longer timeout for first fetch. Kafka consumer seems to do better with poll() with + // longer timeout initially. Looks like it does not handle initial connection setup properly + // with short polls. In my tests it took ~5 seconds before first record was read with this + // hack and 20-30 seconds with out. + long timeout_ms = isFirstFetch ? 4000 : 100; ConsumerRecords records = consumer.poll(timeout_ms); List nonEmpty = Lists.newLinkedList(); - for (PartitionState pState : partitionStates) { - List> pRecords = records.records(pState.topicPartition); - - if (pRecords.size() > 0) { - pState.recordIter = pRecords.iterator(); - nonEmpty.add(pState); + for (PartitionState p : partitionStates) { + p.recordIter = records.records(p.topicPartition).iterator(); + if (p.recordIter.hasNext()) { + nonEmpty.add(p); } - }; + } - // cycle through the partitions in order to interleave records from each of them. + // cycle through the partitions in order to interleave records from each. curBatch = Iterators.cycle(nonEmpty); } @Override public boolean start() throws IOException { - consumer = new KafkaConsumer<>(source.consumerConfig); consumer.assign(source.assignedPartitions); // seek to next offset if consumedOffset is set for(PartitionState p : partitionStates) { if (p.consumedOffset >= 0) { - LOG.info("Reader {} : resuming {} at {}", source.id, p.topicPartition, p.consumedOffset + 1); + LOG.info("{}: resuming {} at {}", name, p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset + 1); } else { - LOG.info("Reader {} : resuming {} at default offset", source.id, p.topicPartition); + LOG.info("{} : resuming {} at default offset", name, p.topicPartition); } } + readNextBatch(true); return advance(); } @@ -488,6 +492,11 @@ public boolean advance() throws IOException { long consumed = pState.consumedOffset; long offset = rawRecord.offset(); + if (curRecord == null) { + LOG.info("{} : first record offset {}", name, offset); // helps with latency to first record + } + + // apply user coders if (consumed >= 0 && offset <= consumed) { // -- (a) // this can happen when compression is enabled in Kafka // should we check if the offset is way off from consumedOffset (say > 1M)? @@ -511,13 +520,10 @@ public boolean advance() throws IOException { curTimestamp = source.timestampFn.apply(curRecord); pState.consumedOffset = offset; - - numRecordsRead++; return true; } else { // -- (b) - - readNextBatch(); + readNextBatch(false); if (!curBatch.hasNext()) { return false; @@ -534,9 +540,10 @@ private static T decode(byte[] bytes, Coder coder) throws IOException { @Override public Instant getWatermark() { - if (numRecordsRead == 0) { + if (curRecord == null) { // haven't read any record yet. what is the right thing to do here? - // may be better to invoke user provided watermarkFn with null and let user decide? + // may be better to invoke watermarkFn with null and let the user decide? + LOG.warn("{} : Returning current time for watermark as no records have been read yet", name); return Instant.now(); } @@ -580,7 +587,7 @@ public void close() throws IOException { } } - // Builder, Source, Reader wrappers when user is only interested in Value in KafkaRecord : + // Builder, Source, and Reader wrappers. Often user is only interested in Value in KafkaRecord : public static class ValueSourceBuilder { From 1872275bcdf292cd94f6a74d8f30ca3f3f77dfbd Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Sat, 27 Feb 2016 12:12:11 -0800 Subject: [PATCH 31/72] minor --- .../google/cloud/dataflow/contrib/kafka/KafkaSource.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 6efa934a80..f59c619da7 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -492,10 +492,6 @@ public boolean advance() throws IOException { long consumed = pState.consumedOffset; long offset = rawRecord.offset(); - if (curRecord == null) { - LOG.info("{} : first record offset {}", name, offset); // helps with latency to first record - } - // apply user coders if (consumed >= 0 && offset <= consumed) { // -- (a) // this can happen when compression is enabled in Kafka @@ -510,6 +506,10 @@ public boolean advance() throws IOException { pState.topicPartition, consumed, offset - consumed - 1); } + if (curRecord == null) { + LOG.info("{} : first record offset {}", name, offset); // helps with latency to first record + } + // apply user coders curRecord = new KafkaRecord( rawRecord.topic(), From ebc7d57fbc988b1a8667ac2285d7ae83465bfc5e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Sat, 27 Feb 2016 12:12:58 -0800 Subject: [PATCH 32/72] minor --- .../com/google/cloud/dataflow/contrib/kafka/KafkaSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index f59c619da7..cb55fb9dc7 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -507,7 +507,7 @@ public boolean advance() throws IOException { } if (curRecord == null) { - LOG.info("{} : first record offset {}", name, offset); // helps with latency to first record + LOG.info("{} : first record offset {}", name, offset); // measure of latency to first record } // apply user coders From 7f7ec7b6d13a62d9ad34d10383df47b954afd5cb Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 29 Feb 2016 12:51:57 -0800 Subject: [PATCH 33/72] move examples to its own directory. mvn dependencies don't work properly yet. --- contrib/examples/kafka/pom.xml | 58 +++++++ .../kafka/examples}/TopHashtagsExample.java | 3 +- contrib/kafka/common-pom.xml | 159 ++++++++++++++++++ contrib/kafka/pom.xml | 151 +---------------- 4 files changed, 227 insertions(+), 144 deletions(-) create mode 100644 contrib/examples/kafka/pom.xml rename contrib/{kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka => examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples}/TopHashtagsExample.java (98%) create mode 100644 contrib/kafka/common-pom.xml diff --git a/contrib/examples/kafka/pom.xml b/contrib/examples/kafka/pom.xml new file mode 100644 index 0000000000..83f9f2dcaa --- /dev/null +++ b/contrib/examples/kafka/pom.xml @@ -0,0 +1,58 @@ + + + + 4.0.0 + + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka-parent + ../../kafka/common-pom.xml + 0.0.1-SNAPSHOT + + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka-examples + Google Cloud Dataflow Kafka Examples + Examples apps using Kafka Source in Google Cloud Dataflow + jar + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka + ${project.version} + + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka-parent + + + + + org.slf4j + slf4j-log4j12 + 1.7.16 + + + log4j + log4j + 1.2.17 + + + diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java similarity index 98% rename from contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java rename to contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index e96caf0120..8765b7a6e2 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -1,4 +1,4 @@ -package com.google.cloud.dataflow.contrib.kafka; +package com.google.cloud.dataflow.contrib.kafka.examples; import java.util.List; import java.util.Map; @@ -17,6 +17,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.cloud.dataflow.contrib.kafka.KafkaSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.Read; diff --git a/contrib/kafka/common-pom.xml b/contrib/kafka/common-pom.xml new file mode 100644 index 0000000000..2c9b00924b --- /dev/null +++ b/contrib/kafka/common-pom.xml @@ -0,0 +1,159 @@ + + + + 4.0.0 + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka-parent + Google Cloud Dataflow Kafka Connectors Parent + Dataflow Library to read Kafka topics + 0.0.1-SNAPSHOT + pom + + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + 0.0.1-SNAPSHOT + [1.2.0,2.0.0) + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + + com.puppycrawl.tools + checkstyle + 6.6 + + + + ../../checkstyle.xml + true + true + true + + + + + check + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.3 + + Google Cloud Dataflow Kafka Contrib + Google Cloud Dataflow Kafka Contrib + com.google.cloud.dataflow.contrib.kafka + false + ]]> + + + + https://cloud.google.com/dataflow/java-sdk/JavaDoc/ + ${basedir}/../../javadoc/dataflow-sdk-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ + ${basedir}/../../javadoc/guava-docs + + + + + + + jar + + package + + + + + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${google-cloud-dataflow-version} + + + + org.apache.kafka + kafka-clients + [0.9,) + + + + + junit + junit + 4.11 + test + + + diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index 15447c953c..b091a5ddeb 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -19,152 +19,17 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 + + com.google.cloud.dataflow + google-cloud-dataflow-java-contrib-kafka-parent + 0.0.1-SNAPSHOT + ./common-pom.xml + + com.google.cloud.dataflow google-cloud-dataflow-java-contrib-kafka Google Cloud Dataflow Kafka Connectors - Library to read Kafka topics from Dataflow - 0.0.1-SNAPSHOT + Dataflow Library to read Kafka topics jar - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - UTF-8 - [1.2.0,2.0.0) - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.2 - - 1.7 - 1.7 - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.12 - - - com.puppycrawl.tools - checkstyle - 6.6 - - - - ../../checkstyle.xml - true - true - true - - - - - check - - - - - - - - org.apache.maven.plugins - maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - Google Cloud Dataflow Kafka Contrib - Google Cloud Dataflow Kafka Contrib - - com.google.cloud.dataflow.contrib.kafka - false - ]]> - - - - https://cloud.google.com/dataflow/java-sdk/JavaDoc/ - ${basedir}/../../javadoc/dataflow-sdk-docs - - - http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ - ${basedir}/../../javadoc/guava-docs - - - - - - - jar - - package - - - - - - - - - com.google.cloud.dataflow - google-cloud-dataflow-java-sdk-all - ${google-cloud-dataflow-version} - - - - org.apache.kafka - kafka-clients - [0.9,) - - - - - org.slf4j - slf4j-log4j12 - 1.7.16 - - - log4j - log4j - 1.2.17 - - - - - junit - junit - 4.11 - test - - From c5cf5ee369ddf4fa0d33d57f07080e6d63ec32d4 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 29 Feb 2016 17:18:28 -0800 Subject: [PATCH 34/72] meven changes for examples directory javadoc and checkstyle fixes invoke watermarkFn even with null record --- contrib/examples/kafka/pom.xml | 168 ++++++++++++-- .../kafka/examples/TopHashtagsExample.java | 67 ++++-- contrib/kafka/common-pom.xml | 159 ------------- contrib/kafka/pom.xml | 138 ++++++++++- .../contrib/kafka/KafkaCheckpointMark.java | 31 ++- .../dataflow/contrib/kafka/KafkaRecord.java | 20 +- .../contrib/kafka/KafkaRecordCoder.java | 63 +++-- .../dataflow/contrib/kafka/KafkaSource.java | 218 +++++++++++------- 8 files changed, 543 insertions(+), 321 deletions(-) delete mode 100644 contrib/kafka/common-pom.xml diff --git a/contrib/examples/kafka/pom.xml b/contrib/examples/kafka/pom.xml index 83f9f2dcaa..f5f39e3f27 100644 --- a/contrib/examples/kafka/pom.xml +++ b/contrib/examples/kafka/pom.xml @@ -19,40 +19,160 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - - com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-kafka-parent - ../../kafka/common-pom.xml - 0.0.1-SNAPSHOT - - com.google.cloud.dataflow google-cloud-dataflow-java-contrib-kafka-examples Google Cloud Dataflow Kafka Examples Examples apps using Kafka Source in Google Cloud Dataflow + 0.0.1-SNAPSHOT jar + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + + com.puppycrawl.tools + checkstyle + 6.6 + + + + ../../../checkstyle.xml + true + true + true + + + + + check + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.3 + + Google Cloud Dataflow Kafka Contrib + Google Cloud Dataflow Kafka Contrib + + com.google.cloud.dataflow.contrib.kafka + false + ]]> + + + + https://cloud.google.com/dataflow/java-sdk/JavaDoc/ + ${basedir}/../../javadoc/dataflow-sdk-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ + ${basedir}/../../javadoc/guava-docs + + + + + + + jar + + package + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.3 + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + + com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-kafka + google-cloud-dataflow-java-contrib-kafka ${project.version} - - - com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-kafka-parent - - - - - org.slf4j - slf4j-log4j12 - 1.7.16 - - - log4j - log4j - 1.2.17 diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 8765b7a6e2..ce59fb1f93 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -1,22 +1,21 @@ -package com.google.cloud.dataflow.contrib.kafka.examples; - -import java.util.List; -import java.util.Map; +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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. + */ -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.StringSerializer; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +package com.google.cloud.dataflow.contrib.kafka.examples; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.ObjectWriter; import com.google.cloud.dataflow.contrib.kafka.KafkaSource; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; @@ -40,6 +39,24 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectWriter; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; + /** * This Dataflow app show cases {@link KafkaSource}. The application reads from a Kafka topic * containing JSON Tweets, calculates top @@ -63,6 +80,9 @@ public class TopHashtagsExample { private static final Logger LOG = LoggerFactory.getLogger(TopHashtagsExample.class); + /** + * Options for the app. + */ public static interface Options extends PipelineOptions { @Description("Sliding window size, in minutes") @Default.Integer(10) @@ -119,7 +139,8 @@ public static void main(String args[]) { .of(Duration.standardMinutes(windowSize)) .every(Duration.standardMinutes(windowPeriod)))) .apply(Count.perElement()) - .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()).withoutDefaults()) + .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()) + .withoutDefaults()) .apply(ParDo.of(new OutputFormatter())) .apply(ParDo.of(new KafkaWriter(options))); @@ -133,7 +154,7 @@ public static void main(String args[]) { private static final ObjectMapper jsonMapper = new ObjectMapper(); /** - * Emit hashtags in the tweet (if any) + * Emit hashtags in the tweet (if any). */ private static class ExtractHashtagsFn extends DoFn { @@ -153,8 +174,8 @@ public void processElement(ProcessContext ctx) throws Exception { @Override public Instant apply(String json) { try { - long timestamp_ms = jsonMapper.readTree(json).path("timestamp_ms").asLong(); - return timestamp_ms == 0 ? Instant.now() : new Instant(timestamp_ms); + long tsMillis = jsonMapper.readTree(json).path("timestamp_ms").asLong(); + return tsMillis == 0 ? Instant.now() : new Instant(tsMillis); } catch (Exception e) { throw Throwables.propagate(e); } @@ -165,8 +186,8 @@ public Instant apply(String json) { private static class OutputFormatter extends DoFn>, String> implements DoFn.RequiresWindowAccess { - transient private DateTimeFormatter formatter; - transient private ObjectWriter jsonWriter; + private transient DateTimeFormatter formatter; + private transient ObjectWriter jsonWriter; static class OutputJson { @JsonProperty String windowStart; diff --git a/contrib/kafka/common-pom.xml b/contrib/kafka/common-pom.xml deleted file mode 100644 index 2c9b00924b..0000000000 --- a/contrib/kafka/common-pom.xml +++ /dev/null @@ -1,159 +0,0 @@ - - - - 4.0.0 - - com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-kafka-parent - Google Cloud Dataflow Kafka Connectors Parent - Dataflow Library to read Kafka topics - 0.0.1-SNAPSHOT - pom - - - - Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - repo - - - - - UTF-8 - 0.0.1-SNAPSHOT - [1.2.0,2.0.0) - - - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.2 - - 1.7 - 1.7 - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 2.12 - - - com.puppycrawl.tools - checkstyle - 6.6 - - - - ../../checkstyle.xml - true - true - true - - - - - check - - - - - - - - org.apache.maven.plugins - maven-source-plugin - 2.4 - - - attach-sources - compile - - jar - - - - attach-test-sources - test-compile - - test-jar - - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - 2.10.3 - - Google Cloud Dataflow Kafka Contrib - Google Cloud Dataflow Kafka Contrib - com.google.cloud.dataflow.contrib.kafka - false - ]]> - - - - https://cloud.google.com/dataflow/java-sdk/JavaDoc/ - ${basedir}/../../javadoc/dataflow-sdk-docs - - - http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ - ${basedir}/../../javadoc/guava-docs - - - - - - - jar - - package - - - - - - - - - com.google.cloud.dataflow - google-cloud-dataflow-java-sdk-all - ${google-cloud-dataflow-version} - - - - org.apache.kafka - kafka-clients - [0.9,) - - - - - junit - junit - 4.11 - test - - - diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml index b091a5ddeb..44ea594b5b 100644 --- a/contrib/kafka/pom.xml +++ b/contrib/kafka/pom.xml @@ -19,17 +19,141 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 - - com.google.cloud.dataflow - google-cloud-dataflow-java-contrib-kafka-parent - 0.0.1-SNAPSHOT - ./common-pom.xml - - com.google.cloud.dataflow google-cloud-dataflow-java-contrib-kafka Google Cloud Dataflow Kafka Connectors Dataflow Library to read Kafka topics + 0.0.1-SNAPSHOT jar + + + Apache License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + repo + + + + + UTF-8 + [1.2.0,2.0.0) + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 2.12 + + + com.puppycrawl.tools + checkstyle + 6.6 + + + + ../../checkstyle.xml + true + true + true + + + + + check + + + + + + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + attach-sources + compile + + jar + + + + attach-test-sources + test-compile + + test-jar + + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.3 + + Google Cloud Dataflow Kafka Contrib + Google Cloud Dataflow Kafka Contrib + + com.google.cloud.dataflow.contrib.kafka + false + ]]> + + + + https://cloud.google.com/dataflow/java-sdk/JavaDoc/ + ${basedir}/../../javadoc/dataflow-sdk-docs + + + http://docs.guava-libraries.googlecode.com/git-history/release18/javadoc/ + ${basedir}/../../javadoc/guava-docs + + + + + + + jar + + package + + + + + + + + + com.google.cloud.dataflow + google-cloud-dataflow-java-sdk-all + ${google-cloud-dataflow-version} + + + + org.apache.kafka + kafka-clients + [0.9,) + + + + + junit + junit + 4.11 + test + + diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java index f945a7e5cd..804c2a6910 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java @@ -1,15 +1,31 @@ -package com.google.cloud.dataflow.contrib.kafka; - -import java.io.IOException; -import java.io.Serializable; -import java.util.List; +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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. + */ -import org.apache.kafka.common.TopicPartition; +package com.google.cloud.dataflow.contrib.kafka; import com.google.cloud.dataflow.sdk.coders.DefaultCoder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.UnboundedSource; +import org.apache.kafka.common.TopicPartition; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; + /** * Checkpoint for an unbounded KafkaSource reader. Consists of Kafka topic name, partition id, * and the latest offset consumed so far. @@ -40,6 +56,9 @@ public void finalizeCheckpoint() throws IOException { */ } + /** + * TopicPartition, offset tuple. Defines specific location in the partitions. + */ public static class PartitionMark implements Serializable { private final TopicPartition topicPartition; private final long offset; diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index 67ee2bc4b3..9ef8137858 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -1,9 +1,27 @@ +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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 com.google.cloud.dataflow.contrib.kafka; import java.io.Serializable; /** - * TODO(rangadi): JavaDoc + * KafkaRecord contains key and value of the record as well as metadata for the record (topic name, + * partition id, and offset). This is essentially a serializable + * {@link org.apache.kafka.clients.consumer.ConsumerRecord}. */ public class KafkaRecord implements Serializable { diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java index 34e27f032a..009916f6fc 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java @@ -1,12 +1,21 @@ -package com.google.cloud.dataflow.contrib.kafka; +/* + * Copyright (C) 2015 Google Inc. + * + * Licensed 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. + */ -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; +package com.google.cloud.dataflow.contrib.kafka; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.CoderException; import com.google.cloud.dataflow.sdk.coders.KvCoder; @@ -17,6 +26,17 @@ import com.google.cloud.dataflow.sdk.util.PropertyNames; import com.google.common.collect.ImmutableList; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; + +/** + * {@link Coder} for {@link KafkaRecord}. + */ public class KafkaRecordCoder extends StandardCoder> { private static final StringUtf8Coder stringCoder = StringUtf8Coder.of(); @@ -46,11 +66,11 @@ public KafkaRecordCoder(Coder keyCoder, Coder valueCoder) { public void encode(KafkaRecord value, OutputStream outStream, Context context) throws CoderException, IOException { Context nested = context.nested(); - stringCoder.encode(value.getTopic(), outStream, nested); - intCoder .encode(value.getPartition(), outStream, nested); - longCoder .encode(value.getOffset(), outStream, nested); - keyCoder .encode(value.getKey(), outStream, nested); - valueCoder .encode(value.getValue(), outStream, nested); + stringCoder.encode(value.getTopic(), outStream, nested); + intCoder.encode(value.getPartition(), outStream, nested); + longCoder.encode(value.getOffset(), outStream, nested); + keyCoder.encode(value.getKey(), outStream, nested); + valueCoder.encode(value.getValue(), outStream, nested); } @Override @@ -58,11 +78,11 @@ public KafkaRecord decode(InputStream inStream, Context context) throws CoderException, IOException { Context nested = context.nested(); return new KafkaRecord( - stringCoder .decode(inStream, nested), - intCoder .decode(inStream, nested), - longCoder .decode(inStream, nested), - keyCoder .decode(inStream, nested), - valueCoder .decode(inStream, nested)); + stringCoder.decode(inStream, nested), + intCoder.decode(inStream, nested), + longCoder.decode(inStream, nested), + keyCoder.decode(inStream, nested), + valueCoder.decode(inStream, nested)); } @Override @@ -79,20 +99,21 @@ public void verifyDeterministic() throws NonDeterministicException { public boolean isRegisterByteSizeObserverCheap(KafkaRecord value, Context context) { return keyCoder.isRegisterByteSizeObserverCheap(value.getKey(), context.nested()) && valueCoder.isRegisterByteSizeObserverCheap(value.getValue(), context.nested()); - //XXX don't we have to implement getEncodedSize()? + //TODO : do we have to implement getEncodedSize()? } @Override - public Object structuralValue(KafkaRecord value) throws Exception{ - if (consistentWithEquals()) + public Object structuralValue(KafkaRecord value) throws Exception { + if (consistentWithEquals()) { return value; - else + } else { return new KafkaRecord( value.getTopic(), value.getPartition(), value.getOffset(), keyCoder.structuralValue(value.getKey()), valueCoder.structuralValue(value.getValue())); + } } @Override diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index cb55fb9dc7..cb8f0f8bfd 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -1,5 +1,5 @@ /* - * Copyright (C) 2015 The Google Inc. + * Copyright (C) 2015 Google Inc. * * Licensed 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 @@ -16,30 +16,6 @@ package com.google.cloud.dataflow.contrib.kafka; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.KafkaConsumer; -import org.apache.kafka.common.PartitionInfo; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.google.api.client.util.Maps; import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; @@ -62,18 +38,37 @@ import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + /** - * TODO(rangadi): JavaDoc + * Dataflow Source for consuming Kafka sources. */ public class KafkaSource { private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); - /* TODO: Overall todos: - * - javadoc at many places - * - confirm non-blocking behavior in advance() - */ - private static class NowTimestampFn implements SerializableFunction { @Override public Instant apply(T input) { @@ -92,28 +87,38 @@ public static Builder unboundedByteSourceBuilder() { } /** - * Similar to {@link #unboundedSourceBuilder()}, except the the source strips KafkaRecord wrapper - * and returns just the value. + * Similar to {@link #unboundedSourceBuilder()}, where user in only interested in value, and + * wants to discard Kafak record key and metadata. + * + * @param value type + * @return {@link ValueSourceBuilder} */ - public static ValueSourceBuilder unboundedValueSourceBuilder() { - return new ValueSourceBuilder( - new Builder() + public static ValueSourceBuilder unboundedValueSourceBuilder() { + return new ValueSourceBuilder( + new Builder() .withKeyCoder(ByteArrayCoder.of())); } + /** + * Builds Unbounded Kafka Source. + * + * @param K key type + * @param V value type + */ public static class Builder { private List topics; - // future: let users specify subset of partitions to read + // future: let users specify subset of partitions to read. private Coder keyCoder; private Coder valueCoder; private SerializableFunction, Instant> timestampFn = new NowTimestampFn<>(); - private Optional, Instant>> watermarkFn = Optional.absent(); + private Optional, Instant>> watermarkFn = + Optional.absent(); private Map mutableConsumerConfig = Maps.newHashMap(); /** - * set of properties that are not required or don't make sense + * set of properties that are not required or don't make sense for our consumer. */ private static final Map ignoredConsumerProperties = ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", @@ -123,18 +128,22 @@ public static class Builder { // lets allow these, applications can have better resume point for restarts. ); - private Builder() { // set config defaults mutableConsumerConfig.putAll(ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", // default to latest offset when last offset is unknown. - "enable.auto.commit", false)); // disable auto commit (may be enabled by the user) + // default to latest offset when we are not resuming. + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", + // disable auto commit of offsets. we don't require group_id. could be enabled by user. + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false)); } /** * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). + * + * @param bootstrapServers Bootstrap servers for Kafka. + * @return Builder */ public Builder withBootstrapServers(String bootstrapServers) { return withConsumerProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); @@ -142,6 +151,9 @@ public Builder withBootstrapServers(String bootstrapServers) { /** * Set Kafka topics to be consumed. This is required. + * + * @param topics topics to read from + * @return Builder */ public Builder withTopics(Collection topics) { this.topics = ImmutableList.copyOf(topics); @@ -149,8 +161,11 @@ public Builder withTopics(Collection topics) { } /** - * Set a {@KafkaConsumer} configuration properties. - * @see ConsumerConfig + * Set a {@link ConsumerConfig} configuration property. + * + * @param configKey configuration property name + * @param configValue value for configuration property + * @return Builder */ public Builder withConsumerProperty(String configKey, Object configValue) { Preconditions.checkArgument(!ignoredConsumerProperties.containsKey(configKey), @@ -162,27 +177,45 @@ public Builder withConsumerProperty(String configKey, Object configValue) /** * Update consumer config properties. Note that this does not not discard already configured. * Same as invoking #withConsumerProperty() with each entry. + * + * @param configUpdate updates to {@link ConsumerConfig} + * @return Builder */ - public Builder withConsumerProperties(Map configToUpdate) { - for(Entry e : configToUpdate.entrySet()) { + public Builder withConsumerProperties(Map configUpdate) { + for (Entry e : configUpdate.entrySet()) { withConsumerProperty(e.getKey(), e.getValue()); } return this; } + /** + * Set Coder for Key. + * + * @param keyCoder Coder for Key + * @return Builder + */ public Builder withKeyCoder(Coder keyCoder) { this.keyCoder = keyCoder; return this; } + /** + * Set Coder for Value. + * + * @param valueCoder Coder for Value + * @return Builder + */ public Builder withValueCoder(Coder valueCoder) { this.valueCoder = valueCoder; return this; } /** - * A function to assign a timestamp to a record. Default is the timestamp when the - * record is processed by {@UnboundedReader#advance()} + * A function to assign a timestamp to a record. When this is not set, processing timestamp + * (when record is processed by {@link UnboundedReader#advance()}) is used. + * + * @param timestampFn Function to calculate timestamp of a record. + * @return Builder */ public Builder withTimestampFn( SerializableFunction, Instant> timestampFn) { @@ -192,13 +225,24 @@ public Builder withTimestampFn( /** * A function to calculate watermark. When this is not set, last record timestamp is returned - * in {@link UnboundedReader#getWatermark()}. + * in {@link UnboundedReader#getWatermark()}. NOTE: this might be invoked before any records + * have been read, in which case the function is invoked with null KafkaRecord. User could + * decide the policy in such a case (e.g. return current timestamp). + * + * @param watermarkFn Function to calculate watermark for a record. NOTE: input might be null. + * @return Builder */ - public Builder withWatermarkFn(SerializableFunction, Instant> watermarkFn) { + public Builder withWatermarkFn( + SerializableFunction, Instant> watermarkFn) { this.watermarkFn = Optional.of(watermarkFn); return this; } + /** + * Build Unbounded Kafka Source + * + * @return UnboundedKafkaSource + */ public UnboundedSource, KafkaCheckpointMark> build() { ImmutableMap consumerConfig = ImmutableMap.copyOf(mutableConsumerConfig); @@ -219,12 +263,12 @@ public UnboundedSource, KafkaCheckpointMark> build() { valueCoder, timestampFn, watermarkFn, - ImmutableList.of() // no assigned partitions yet + ImmutableList.of() // no assigned partitions yet. ); } } - /** Static class, prevent instantiation */ + /** Static class, prevent instantiation. */ private KafkaSource() {} private static class UnboundedKafkaSource @@ -300,17 +344,17 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { List> assignments = Lists.newArrayList(); - for (int i=0; inewArrayList()); } - for (int i=0; i> result = Lists.newArrayList(); - for (int i=0; i assignedToSplit = assignments.get(i); LOG.info("Partitions assigned to split {} (total {}): {}", @@ -415,7 +459,7 @@ public PartitionState apply(TopicPartition tp) { "checkPointMark and assignedPartitions should match"); // we could consider allowing a mismatch, though it is not expected in current Dataflow - for (int i=0; i < source.assignedPartitions.size(); i++) { + for (int i = 0; i < source.assignedPartitions.size(); i++) { KafkaCheckpointMark.PartitionMark ckptMark = checkpointMark.getPartitions().get(i); TopicPartition assigned = source.assignedPartitions.get(i); @@ -435,8 +479,8 @@ private void readNextBatch(boolean isFirstFetch) { // longer timeout initially. Looks like it does not handle initial connection setup properly // with short polls. In my tests it took ~5 seconds before first record was read with this // hack and 20-30 seconds with out. - long timeout_ms = isFirstFetch ? 4000 : 100; - ConsumerRecords records = consumer.poll(timeout_ms); + long timeoutMillis = isFirstFetch ? 4000 : 100; + ConsumerRecords records = consumer.poll(timeoutMillis); List nonEmpty = Lists.newLinkedList(); @@ -457,7 +501,7 @@ public boolean start() throws IOException { consumer.assign(source.assignedPartitions); // seek to next offset if consumedOffset is set - for(PartitionState p : partitionStates) { + for (PartitionState p : partitionStates) { if (p.consumedOffset >= 0) { LOG.info("{}: resuming {} at {}", name, p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset + 1); @@ -475,7 +519,7 @@ public boolean advance() throws IOException { /* Read first record (if any). we need to loop here because : * - (a) some records initially need to be skipped since they are before consumedOffset * - (b) when the current batch empty, we want to readNextBatch() and then advance. - * - (c) curBatch is an iterator of iterators and we want to interleave the records from each. + * - (c) curBatch is an iterator of iterators. we interleave the records from each. * curBatch.next() might return an empty iterator. */ while (true) { @@ -507,7 +551,7 @@ public boolean advance() throws IOException { } if (curRecord == null) { - LOG.info("{} : first record offset {}", name, offset); // measure of latency to first record + LOG.info("{} : first record offset {}", name, offset); } // apply user coders @@ -533,30 +577,31 @@ public boolean advance() throws IOException { } private static T decode(byte[] bytes, Coder coder) throws IOException { - if (bytes == null) + if (bytes == null) { return null; // is this the right thing to do? + } return coder.decode(new ExposedByteArrayInputStream(bytes), Coder.Context.OUTER); } @Override public Instant getWatermark() { if (curRecord == null) { - // haven't read any record yet. what is the right thing to do here? - // may be better to invoke watermarkFn with null and let the user decide? - LOG.warn("{} : Returning current time for watermark as no records have been read yet", name); - return Instant.now(); + LOG.warn("{} : getWatermar() : no records have been read yet.", name); + // if watermarkFn is set, we invoke it with null record and let the user decide what to do, + // otherwise return now(). } - if (source.watermarkFn.isPresent()) + if (source.watermarkFn.isPresent()) { return source.watermarkFn.get().apply(curRecord); - else - return curTimestamp; + } else { + return curRecord == null ? Instant.now() : curTimestamp; + } } @Override public CheckpointMark getCheckpointMark() { - return new KafkaCheckpointMark( - ImmutableList.copyOf(Lists.transform(partitionStates, // avoid lazy (consumedOffset can change) + return new KafkaCheckpointMark(ImmutableList.copyOf(// avoid lazy (consumedOffset can change) + Lists.transform(partitionStates, new Function() { public PartitionMark apply(PartitionState p) { return new PartitionMark(p.topicPartition, p.consumedOffset); @@ -589,6 +634,10 @@ public void close() throws IOException { // Builder, Source, and Reader wrappers. Often user is only interested in Value in KafkaRecord : + /** + * Builder for Kafka Source where user is not interested in Kafka metadata and key for a record, + * but just the value. + */ public static class ValueSourceBuilder { private Builder underlying; @@ -624,7 +673,12 @@ public ValueSourceBuilder withTimestampFn(SerializableFunction ti /** * A function to calculate watermark. When this is not set, last record timestamp is returned - * in {@link UnboundedReader#getWatermark()}. + * in {@link UnboundedReader#getWatermark()}. NOTE: this might be invoked before any records + * have been read, in which case the function is invoked with null value. User could + * decide the policy in such a case (e.g. return current timestamp). + * + * @param watermarkFn Function to calculate watermark for a record. NOTE: input might be null. + * @return Builder */ public ValueSourceBuilder withWatermarkFn(SerializableFunction watermarkFn) { return new ValueSourceBuilder( @@ -635,11 +689,12 @@ public UnboundedSource build() { return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); } - private static - SerializableFunction, O> unwrapKafkaAndThen(final SerializableFunction fn) { - return new SerializableFunction, O>() { - public O apply(KafkaRecord record) { - return fn.apply(record.getValue()); + private static + SerializableFunction, OutT> unwrapKafkaAndThen( + final SerializableFunction fn) { + return new SerializableFunction, OutT>() { + public OutT apply(KafkaRecord record) { + return fn.apply(record == null ? null : record.getValue()); } }; } @@ -649,7 +704,8 @@ public O apply(KafkaRecord record) { * Usually the users are only interested in value in KafkaRecord. This is a convenient class * to strip out other fields in KafkaRecord returned by UnboundedKafkaValueSource */ - private static class UnboundedKafkaValueSource extends UnboundedSource { + private static class UnboundedKafkaValueSource + extends UnboundedSource { private final UnboundedKafkaSource underlying; @@ -669,8 +725,10 @@ public UnboundedKafkaValueSource apply(UnboundedKafkaSource input) } @Override - public UnboundedReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { - return new UnboundedKafkaValueReader(this, underlying.createReader(options, checkpointMark)); + public UnboundedReader createReader( + PipelineOptions options, KafkaCheckpointMark checkpointMark) { + return new UnboundedKafkaValueReader(this, + underlying.createReader(options, checkpointMark)); } @Override From 16182eee38adc5458ec83b634be0d4097c6b81e1 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 29 Feb 2016 17:29:04 -0800 Subject: [PATCH 35/72] javadoc update --- .../dataflow/contrib/kafka/KafkaSource.java | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index cb8f0f8bfd..33a380dfc5 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -64,6 +64,22 @@ /** * Dataflow Source for consuming Kafka sources. + * + *
+ * Usage:
+ *        UnboundedSource kafkaSource = KafkaSource
+ *            .<String>unboundedValueSourceBuilder()
+ *            .withBootstrapServers("broker_1:9092,broker_2:9092)
+ *            .withTopics(ImmutableList.of("topic_a", "topic_b")
+ *            .withValueCoder(StringUtf8Coder.of())
+ *            .withTimestampFn(timestampFn)
+ *            .withWatermarkFn(watermarkFn)
+ *            .build();
+ *
+ *        pipeline
+ *          .apply(Read.from(kafkaSource).named("read_topic_a_and_b"))
+ *          ....
+ * 
*/ public class KafkaSource { @@ -554,7 +570,7 @@ public boolean advance() throws IOException { LOG.info("{} : first record offset {}", name, offset); } - // apply user coders + // apply user coders. might want to allow skipping records that fail in coders. curRecord = new KafkaRecord( rawRecord.topic(), rawRecord.partition(), From 025f047412093727444a49944cbab18c46e85c7f Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 1 Mar 2016 13:21:33 -0800 Subject: [PATCH 36/72] watermark should be Instant(Long.MIN_VALUE) when the reader hasn't read any records. --- .../dataflow/contrib/kafka/KafkaSource.java | 53 ++++++++----------- 1 file changed, 21 insertions(+), 32 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 33a380dfc5..b99a209128 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -67,7 +67,7 @@ * *
  * Usage:
- *        UnboundedSource kafkaSource = KafkaSource
+ *        UnboundedSource<String, ?> kafkaSource = KafkaSource
  *            .<String>unboundedValueSourceBuilder()
  *            .withBootstrapServers("broker_1:9092,broker_2:9092)
  *            .withTopics(ImmutableList.of("topic_a", "topic_b")
@@ -241,11 +241,9 @@ public Builder withTimestampFn(
 
     /**
      * A function to calculate watermark. When this is not set, last record timestamp is returned
-     * in {@link UnboundedReader#getWatermark()}. NOTE: this might be invoked before any records
-     * have been read, in which case the function is invoked with null KafkaRecord. User could
-     * decide the policy in such a case (e.g. return current timestamp).
+     * in {@link UnboundedReader#getWatermark()}.
      *
-     * @param watermarkFn Function to calculate watermark for a record. NOTE: input might be null.
+     * @param watermarkFn to calculate watermark at a record.
      * @return Builder
      */
     public Builder withWatermarkFn(
@@ -420,18 +418,11 @@ public Coder> getDefaultOutputCoder() {
   private static class UnboundedKafkaReader
              extends UnboundedReader> {
 
-    private final UnboundedKafkaSource source;
-    private final String name;
-    private KafkaConsumer consumer;
-
-    // maintains state of each assigned partition
+    // maintains state of each assigned partition (buffered records and consumed offset)
     private static class PartitionState {
       private final TopicPartition topicPartition;
-
-      private Iterator> recordIter = Iterators.emptyIterator();
-
       private long consumedOffset;
-      // might need to keep track of per partition watermark. not decided yet about the semantics
+      private Iterator> recordIter = Iterators.emptyIterator();
 
       PartitionState(TopicPartition partition, long offset) {
         this.topicPartition = partition;
@@ -439,13 +430,17 @@ private static class PartitionState {
       }
     }
 
-    private List partitionStates;
-
+    private final UnboundedKafkaSource source;
+    private final String name;
+    private KafkaConsumer consumer;
+    private final List partitionStates;
     private KafkaRecord curRecord;
     private Instant curTimestamp;
-
     private Iterator curBatch = Iterators.emptyIterator();
 
+    /** watermark before any records have been read. */
+    private static Instant initialWatermark = new Instant(Long.MIN_VALUE);
+
     public String toString() {
       return name;
     }
@@ -476,7 +471,7 @@ public PartitionState apply(TopicPartition tp) {
         // we could consider allowing a mismatch, though it is not expected in current Dataflow
 
         for (int i = 0; i < source.assignedPartitions.size(); i++) {
-          KafkaCheckpointMark.PartitionMark ckptMark = checkpointMark.getPartitions().get(i);
+          PartitionMark ckptMark = checkpointMark.getPartitions().get(i);
           TopicPartition assigned = source.assignedPartitions.get(i);
 
           Preconditions.checkState(ckptMark.getTopicPartition().equals(assigned),
@@ -602,16 +597,12 @@ private static  T decode(byte[] bytes, Coder coder) throws IOException {
     @Override
     public Instant getWatermark() {
       if (curRecord == null) {
-        LOG.warn("{} : getWatermar() : no records have been read yet.", name);
-        // if watermarkFn is set, we invoke it with null record and let the user decide what to do,
-        // otherwise return now().
+        LOG.warn("{} : getWatermark() : no records have been read yet.", name);
+        return initialWatermark;
       }
 
-      if (source.watermarkFn.isPresent()) {
-        return source.watermarkFn.get().apply(curRecord);
-      } else {
-        return curRecord == null ? Instant.now() : curTimestamp;
-      }
+      return source.watermarkFn.isPresent() ?
+          source.watermarkFn.get().apply(curRecord) : curTimestamp;
     }
 
     @Override
@@ -633,7 +624,7 @@ public PartitionMark apply(PartitionState p) {
 
     @Override
     public KafkaRecord getCurrent() throws NoSuchElementException {
-      // TODO: should we delay updating consumed offset till now?
+      // should we delay updating consumed offset till this point? Mostly not required.
       return curRecord;
     }
 
@@ -689,11 +680,9 @@ public ValueSourceBuilder withTimestampFn(SerializableFunction ti
 
     /**
      * A function to calculate watermark. When this is not set, last record timestamp is returned
-     * in {@link UnboundedReader#getWatermark()}. NOTE: this might be invoked before any records
-     * have been read, in which case the function is invoked with null value. User could
-     * decide the policy in such a case (e.g. return current timestamp).
+     * in {@link UnboundedReader#getWatermark()}.
      *
-     * @param watermarkFn Function to calculate watermark for a record. NOTE: input might be null.
+     * @param watermarkFn to calculate watermark at a record.
      * @return Builder
      */
     public ValueSourceBuilder withWatermarkFn(SerializableFunction watermarkFn) {
@@ -710,7 +699,7 @@ SerializableFunction, OutT> unwrapKafkaAndThen(
         final SerializableFunction fn) {
       return new SerializableFunction, OutT>() {
         public OutT apply(KafkaRecord record) {
-          return fn.apply(record == null ? null : record.getValue());
+          return fn.apply(record.getValue());
         }
       };
     }

From cb20dc946bbc0823994ca587e322cd84247b238b Mon Sep 17 00:00:00 2001
From: Raghu Angadi 
Date: Tue, 1 Mar 2016 23:30:59 -0800
Subject: [PATCH 37/72] Factory method for creating consumer.

---
 .../dataflow/contrib/kafka/KafkaSource.java   | 49 +++++++++++++++++--
 1 file changed, 44 insertions(+), 5 deletions(-)

diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
index b99a209128..e34f00c229 100644
--- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
+++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
@@ -38,6 +38,7 @@
 import com.google.common.collect.Lists;
 import com.google.common.io.Closeables;
 
+import org.apache.kafka.clients.consumer.Consumer;
 import org.apache.kafka.clients.consumer.ConsumerConfig;
 import org.apache.kafka.clients.consumer.ConsumerRecord;
 import org.apache.kafka.clients.consumer.ConsumerRecords;
@@ -109,7 +110,7 @@ public static Builder unboundedByteSourceBuilder() {
    * @param  value type
    * @return {@link ValueSourceBuilder}
    */
-  public static  ValueSourceBuilder unboundedValueSourceBuilder() {
+  public static  ValueSourceBuilder unboundedValueSourceBuilder() {
     return new ValueSourceBuilder(
        new Builder()
        .withKeyCoder(ByteArrayCoder.of()));
@@ -130,6 +131,13 @@ public static class Builder {
     private SerializableFunction, Instant> timestampFn = new NowTimestampFn<>();
     private Optional, Instant>> watermarkFn =
         Optional.absent();
+    private SerializableFunction, Consumer>
+      kafkaConsumerFactoryFn =
+        new SerializableFunction, Consumer>() {
+          public Consumer apply(Map config) {
+            return new KafkaConsumer<>(config); // default 0.9 consumer
+          }
+        };
 
     private Map mutableConsumerConfig = Maps.newHashMap();
 
@@ -252,6 +260,18 @@ public Builder withWatermarkFn(
       return this;
     }
 
+    /**
+     * A factory to create Kafka {@link Consumer} from consumer configuration.
+     * Mainly used for tests.
+     * @param kafkaConsumerFactoryFn function to create
+     * @return
+     */
+    public Builder withKafkaConsumerFactoryFn(
+      SerializableFunction, Consumer> kafkaConsumerFactoryFn) {
+      this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn;
+      return this;
+    }
+
     /**
      * Build Unbounded Kafka Source
      *
@@ -277,6 +297,7 @@ public UnboundedSource, KafkaCheckpointMark> build() {
           valueCoder,
           timestampFn,
           watermarkFn,
+          kafkaConsumerFactoryFn,
           ImmutableList.of() // no assigned partitions yet.
           );
     }
@@ -296,6 +317,8 @@ private static class UnboundedKafkaSource
     private final SerializableFunction, Instant> timestampFn;
     // would it be a good idea to pass currentTimestamp to watermarkFn?
     private final Optional, Instant>> watermarkFn;
+    private
+      SerializableFunction, Consumer> kafkaConsumerFactoryFn;
     private final List assignedPartitions;
 
     public UnboundedKafkaSource(
@@ -306,6 +329,7 @@ public UnboundedKafkaSource(
         Coder valueCoder,
         SerializableFunction, Instant> timestampFn,
         Optional, Instant>> watermarkFn,
+        SerializableFunction, Consumer> kafkaConsumerFactoryFn,
         List assignedPartitions) {
 
       this.id = id;
@@ -315,6 +339,7 @@ public UnboundedKafkaSource(
       this.valueCoder = valueCoder;
       this.timestampFn = timestampFn;
       this.watermarkFn = watermarkFn;
+      this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn;
       this.assignedPartitions = ImmutableList.copyOf(assignedPartitions);
     }
 
@@ -322,7 +347,7 @@ public UnboundedKafkaSource(
     public List> generateInitialSplits(
         int desiredNumSplits, PipelineOptions options) throws Exception {
 
-      KafkaConsumer consumer = new KafkaConsumer(consumerConfig);
+      Consumer consumer = kafkaConsumerFactoryFn.apply(consumerConfig);
 
       List partitions = Lists.newArrayList();
 
@@ -382,6 +407,7 @@ public int compare(TopicPartition tp1, TopicPartition tp2) {
             this.valueCoder,
             this.timestampFn,
             this.watermarkFn,
+            this.kafkaConsumerFactoryFn,
             assignedToSplit));
       }
 
@@ -432,7 +458,7 @@ private static class PartitionState {
 
     private final UnboundedKafkaSource source;
     private final String name;
-    private KafkaConsumer consumer;
+    private Consumer consumer;
     private final List partitionStates;
     private KafkaRecord curRecord;
     private Instant curTimestamp;
@@ -488,7 +514,8 @@ private void readNextBatch(boolean isFirstFetch) {
 
       // Use a longer timeout for first fetch. Kafka consumer seems to do better with poll() with
       // longer timeout initially. Looks like it does not handle initial connection setup properly
-      // with short polls. In my tests it took ~5 seconds before first record was read with this
+      // with short polls and backoff policy in Dataflow might be making things worse for
+      // this case. In my tests it took ~5 seconds before first record was read with this
       // hack and 20-30 seconds with out.
       long timeoutMillis = isFirstFetch ? 4000 : 100;
       ConsumerRecords records = consumer.poll(timeoutMillis);
@@ -508,7 +535,7 @@ private void readNextBatch(boolean isFirstFetch) {
 
     @Override
     public boolean start() throws IOException {
-      consumer = new KafkaConsumer<>(source.consumerConfig);
+      consumer = source.kafkaConsumerFactoryFn.apply(source.consumerConfig);
       consumer.assign(source.assignedPartitions);
 
       // seek to next offset if consumedOffset is set
@@ -690,6 +717,18 @@ public ValueSourceBuilder withWatermarkFn(SerializableFunction wa
           underlying.withTimestampFn(unwrapKafkaAndThen(watermarkFn)));
     }
 
+    /**
+     * A factory to create Kafka {@link Consumer} from consumer configuration.
+     * Mainly used for tests.
+     * @param kafkaConsumerFactoryFn function to create
+     * @return
+     */
+    public ValueSourceBuilder withKafkaConsumerFactoryFn(
+      SerializableFunction, Consumer> kafkaConsumerFactoryFn) {
+      return new ValueSourceBuilder(
+          underlying.withKafkaConsumerFactoryFn(kafkaConsumerFactoryFn));
+    }
+
     public UnboundedSource build() {
       return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build());
     }

From 29e8c0ae7306ad43a8b2eea75375fb6abf1b0fa6 Mon Sep 17 00:00:00 2001
From: Raghu Angadi 
Date: Wed, 2 Mar 2016 01:24:40 -0800
Subject: [PATCH 38/72] unit tests

---
 contrib/kafka/pom.xml                         |  21 +-
 .../dataflow/contrib/kafka/KafkaSource.java   |  11 +
 .../contrib/kafka/KafkaSourceTest.java        | 310 ++++++++++++++++++
 3 files changed, 340 insertions(+), 2 deletions(-)
 create mode 100644 contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java

diff --git a/contrib/kafka/pom.xml b/contrib/kafka/pom.xml
index 44ea594b5b..7fe8165bc8 100644
--- a/contrib/kafka/pom.xml
+++ b/contrib/kafka/pom.xml
@@ -37,6 +37,9 @@
   
     UTF-8
     [1.2.0,2.0.0)
+    1.3
+    4.11
+    1.7.7
   
 
   
@@ -148,11 +151,25 @@
       [0.9,)
     
 
-    
+    
+    
+      org.hamcrest
+      hamcrest-all
+      ${hamcrest.version}
+      test
+    
+
     
       junit
       junit
-      4.11
+      ${junit.version}
+      test
+    
+
+    
+      org.slf4j
+      slf4j-jdk14
+      ${slf4j.version}
       test
     
   
diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
index e34f00c229..5af124e930 100644
--- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
+++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
@@ -31,6 +31,7 @@
 import com.google.common.base.Joiner;
 import com.google.common.base.Optional;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ComparisonChain;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -417,6 +418,16 @@ public int compare(TopicPartition tp1, TopicPartition tp2) {
     @Override
     public UnboundedKafkaReader createReader(PipelineOptions options,
                                                    KafkaCheckpointMark checkpointMark) {
+      if (assignedPartitions.isEmpty()) {
+        LOG.warn("hack: working around DirectRunner issue. It does not generateSplits()");
+        // generate single split and return reader from it.
+        try {
+          return new UnboundedKafkaReader(
+              generateInitialSplits(1, options).get(0), checkpointMark);
+        } catch (Exception e) {
+          Throwables.propagate(e);
+        }
+      }
       return new UnboundedKafkaReader(this, checkpointMark);
     }
 
diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java
new file mode 100644
index 0000000000..c54cf63e0b
--- /dev/null
+++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java
@@ -0,0 +1,310 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * Licensed 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 com.google.cloud.dataflow.contrib.kafka;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.testing.DataflowAssert;
+import com.google.cloud.dataflow.sdk.testing.RunnableOnService;
+import com.google.cloud.dataflow.sdk.testing.TestPipeline;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.Max;
+import com.google.cloud.dataflow.sdk.transforms.Min;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.transforms.SerializableFunction;
+import com.google.cloud.dataflow.sdk.util.CoderUtils;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.MockConsumer;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.annotation.Nullable;
+
+/**
+ * Tests of {@link KafkaSource}.
+ */
+@RunWith(JUnit4.class)
+public class KafkaSourceTest {
+  /*
+   * The tests below borrow code and structure from CountingSourceTest. In addition verifies
+   * the reader interleaves the records from multiple partitions.
+   *
+   * Other tests to consider :
+   *   - test KafkaRecordCoder
+   */
+
+  // Update mock consumer with records distributed among the given topics, each with given number
+  // of partitions. Records are assigned in round-robin order among the partitions.
+  private static MockConsumer mkMockConsumer(
+      List topics, int partitionsPerTopic, int numElements) {
+
+    final List partitions = new ArrayList<>();
+    final Map>> records = new HashMap<>();
+    Map> partitionMap = new HashMap<>();
+
+    for (String topic : topics) {
+      List partIds = Lists.newArrayListWithCapacity(partitionsPerTopic);
+      for (int i = 0; i < partitionsPerTopic; i++) {
+        partitions.add(new TopicPartition(topic, i));
+        partIds.add(new PartitionInfo(topic, i, null, null, null));
+      }
+      partitionMap.put(topic, partIds);
+    }
+
+    int numPartitions = partitions.size();
+    long[] offsets = new long[numPartitions];
+
+    for (int i = 0; i < numElements; i++) {
+      int pIdx = i % numPartitions;
+      TopicPartition tp = partitions.get(pIdx);
+
+      if (!records.containsKey(tp)) {
+        records.put(tp, new ArrayList>());
+      }
+      records.get(tp).add(
+          new ConsumerRecord(
+              tp.topic(),
+              tp.partition(),
+              offsets[pIdx]++,
+              null, // key
+              ByteBuffer.wrap(new byte[8]).putLong(i).array())); // value is 8 byte record id.
+    }
+
+    MockConsumer consumer =
+        new MockConsumer(OffsetResetStrategy.EARLIEST) {
+          // override assign() to add records that belong to the assigned partitions.
+          public void assign(List assigned) {
+            super.assign(assigned);
+            for (TopicPartition tp : assigned) {
+              for (ConsumerRecord r : records.get(tp)) {
+                addRecord(r);
+              }
+              seek(tp, 0);
+            }
+          }
+        };
+
+    for (String topic : topics) {
+      consumer.updatePartitions(topic, partitionMap.get(topic));
+    }
+
+    return consumer;
+  }
+
+  private static class ConsumerFactoryFn
+                implements SerializableFunction, Consumer> {
+    private final List topics;
+    private final int partitionsPerTopic;
+    private final int numElements;
+
+    public ConsumerFactoryFn(List topics, int partitionsPerTopic, int numElements) {
+      this.topics = topics;
+      this.partitionsPerTopic = partitionsPerTopic;
+      this.numElements = numElements;
+    }
+
+    public Consumer apply(Map config) {
+      return mkMockConsumer(topics, partitionsPerTopic, numElements);
+    }
+  }
+
+  /**
+   * Creates a consumer with two topics, with 5 partitions each.
+   * numElements are (round-robin) assigned all the 10 partitions.
+   */
+  private static UnboundedSource mkKafkaSource(
+      int numElements,
+      @Nullable SerializableFunction timestampFn) {
+
+    List topics = ImmutableList.of("topic_a", "topic_b");
+
+    KafkaSource.ValueSourceBuilder builder = KafkaSource.unboundedValueSourceBuilder()
+        .withBootstrapServers("none")
+        .withTopics(topics)
+        .withKafkaConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions
+        .withValueCoder(BigEndianLongCoder.of());
+
+    if (timestampFn != null) {
+      builder = builder.withTimestampFn(timestampFn);
+    }
+
+    return builder.build();
+  }
+
+  public static void addCountingAsserts(PCollection input, long numElements) {
+    // Count == numElements
+    DataflowAssert
+      .thatSingleton(input.apply("Count", Count.globally()))
+      .isEqualTo(numElements);
+    // Unique count == numElements
+    DataflowAssert
+      .thatSingleton(input.apply(RemoveDuplicates.create())
+                          .apply("UniqueCount", Count.globally()))
+      .isEqualTo(numElements);
+    // Min == 0
+    DataflowAssert
+      .thatSingleton(input.apply("Min", Min.globally()))
+      .isEqualTo(0L);
+    // Max == numElements-1
+    DataflowAssert
+      .thatSingleton(input.apply("Max", Max.globally()))
+      .isEqualTo(numElements - 1);
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUnboundedSource() {
+    Pipeline p = TestPipeline.create();
+    int numElements = 1000;
+
+    PCollection input = p.apply(Read
+            .from(mkKafkaSource(numElements, null))
+            .withMaxNumRecords(numElements));
+
+    addCountingAsserts(input, numElements);
+    p.run();
+  }
+
+  private static class ElementValueDiff extends DoFn {
+    @Override
+    public void processElement(ProcessContext c) throws Exception {
+      c.output(c.element() - c.timestamp().getMillis());
+    }
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUnboundedSourceTimestamps() {
+    Pipeline p = TestPipeline.create();
+    int numElements = 1000;
+
+    PCollection input = p.apply(Read
+        .from(mkKafkaSource(numElements, new ValueAsTimestampFn()))
+            .withMaxNumRecords(numElements));
+    addCountingAsserts(input, numElements);
+
+    PCollection diffs = input
+        .apply("TimestampDiff", ParDo.of(new ElementValueDiff()))
+        .apply("RemoveDuplicateTimestamps", RemoveDuplicates.create());
+    // This assert also confirms that diffs only has one unique value.
+    DataflowAssert.thatSingleton(diffs).isEqualTo(0L);
+
+    p.run();
+  }
+
+  @Test
+  @Category(RunnableOnService.class)
+  public void testUnboundedSourceSplits() throws Exception {
+    Pipeline p = TestPipeline.create();
+    int numElements = 1000;
+    int numSplits = 10;
+
+    UnboundedSource initial = mkKafkaSource(numElements, null);
+    List> splits =
+        initial.generateInitialSplits(numSplits, p.getOptions());
+    assertEquals("Expected exact splitting", numSplits, splits.size());
+
+    long elementsPerSplit = numElements / numSplits;
+    assertEquals("Expected even splits", numElements, elementsPerSplit * numSplits);
+    PCollectionList pcollections = PCollectionList.empty(p);
+    for (int i = 0; i < splits.size(); ++i) {
+      pcollections = pcollections.and(
+          p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit)));
+    }
+    PCollection input = pcollections.apply(Flatten.pCollections());
+
+    addCountingAsserts(input, numElements);
+    p.run();
+  }
+
+  /**
+   * A timestamp function that uses the given value as the timestamp.
+   */
+  private static class ValueAsTimestampFn implements SerializableFunction {
+    @Override
+    public Instant apply(Long input) {
+      return new Instant(input);
+    }
+  }
+
+  @Test
+  public void testUnboundedSourceCheckpointMark() throws Exception {
+    int numElements = 85; // make sure some partitions have more records than other
+
+    // create a single split:
+    UnboundedSource source =
+        mkKafkaSource(numElements, new ValueAsTimestampFn())
+          .generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create())
+          .get(0);
+
+    UnboundedReader reader = source.createReader(null, null);
+    final int numToSkip = 3;
+    // advance once:
+    assertTrue(reader.start());
+
+    // Advance the source numToSkip-1 elements and manually save state.
+    for (long l = 0; l < numToSkip-1; ++l) {
+      reader.advance();
+    }
+
+    // Confirm that we get the expected element in sequence before checkpointing.
+    assertEquals(numToSkip-1, (long) reader.getCurrent());
+    assertEquals(numToSkip-1, reader.getCurrentTimestamp().getMillis());
+
+    // Checkpoint and restart, and confirm that the source continues correctly.
+    KafkaCheckpointMark mark = CoderUtils.clone(
+        source.getCheckpointMarkCoder(), (KafkaCheckpointMark) reader.getCheckpointMark());
+    reader = source.createReader(null, mark);
+    assertTrue(reader.start());
+
+    // Confirm that we get the next elements in sequence.
+    // This also confirms that Reader interleaves records from each partitions by the reader.
+    for (int i = numToSkip; i < numElements; i++) {
+      assertEquals(i, (long) reader.getCurrent());
+      assertEquals(i, reader.getCurrentTimestamp().getMillis());
+      reader.advance();
+    }
+  }
+}

From a14a9eabb5fea42dfb27167ea07bcec8a257ca24 Mon Sep 17 00:00:00 2001
From: Raghu Angadi 
Date: Wed, 2 Mar 2016 10:26:46 -0800
Subject: [PATCH 39/72] minor

---
 .../cloud/dataflow/contrib/kafka/KafkaSource.java  | 14 +++++---------
 1 file changed, 5 insertions(+), 9 deletions(-)

diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
index 5af124e930..90cf9746ef 100644
--- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
+++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java
@@ -264,8 +264,6 @@ public Builder withWatermarkFn(
     /**
      * A factory to create Kafka {@link Consumer} from consumer configuration.
      * Mainly used for tests.
-     * @param kafkaConsumerFactoryFn function to create
-     * @return
      */
     public Builder withKafkaConsumerFactoryFn(
       SerializableFunction, Consumer> kafkaConsumerFactoryFn) {
@@ -354,7 +352,7 @@ public List> generateInitialSplits(
 
       // fetch partitions for each topic
       // sort by 
-      // round-robin assign the partition to splits
+      // round-robin assign the partitions to splits
 
       try {
         for (String topic : topics) {
@@ -496,11 +494,9 @@ public PartitionState apply(TopicPartition tp) {
             }
         }));
 
-      // a) verify that assigned and check-pointed partitions match exactly
-      // b) set consumed offsets
-
       if (checkpointMark != null) {
-        // set consumed offset
+        // a) verify that assigned and check-pointed partitions match exactly
+        // b) set consumed offsets
 
         Preconditions.checkState(
             checkpointMark.getPartitions().size() == source.assignedPartitions.size(),
@@ -549,7 +545,7 @@ public boolean start() throws IOException {
       consumer = source.kafkaConsumerFactoryFn.apply(source.consumerConfig);
       consumer.assign(source.assignedPartitions);
 
-      // seek to next offset if consumedOffset is set
+      // seek to consumedOffset + 1 if it is set
       for (PartitionState p : partitionStates) {
         if (p.consumedOffset >= 0) {
           LOG.info("{}: resuming {} at {}", name, p.topicPartition, p.consumedOffset + 1);
@@ -566,7 +562,7 @@ public boolean start() throws IOException {
     @Override
     public boolean advance() throws IOException {
       /* Read first record (if any). we need to loop here because :
-       *  - (a) some records initially need to be skipped since they are before consumedOffset
+       *  - (a) some records initially need to be skipped if they are consumedOffset
        *  - (b) when the current batch empty, we want to readNextBatch() and then advance.
        *  - (c) curBatch is an iterator of iterators. we interleave the records from each.
        *    curBatch.next() might return an empty iterator.

From 73691b76a5650858f296e902590a8577765754db Mon Sep 17 00:00:00 2001
From: Raghu Angadi 
Date: Wed, 2 Mar 2016 18:39:06 -0800
Subject: [PATCH 40/72] fix serialization issue

---
 .../kafka/examples/TopHashtagsExample.java    |  5 ++--
 .../dataflow/contrib/kafka/KafkaSource.java   | 28 +++++++++----------
 .../contrib/kafka/KafkaSourceTest.java        | 14 ++++++----
 3 files changed, 25 insertions(+), 22 deletions(-)

diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java
index ce59fb1f93..99a29e1d83 100644
--- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java
+++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java
@@ -69,12 +69,11 @@
  *          --project=GCP_PROJECT                                           \
  *          --stagingLocation=GS_STAGING_DIRECTORY                          \
  *          --runner=BlockingDataflowPipelineRunner                         \
- *          --bootstrapServers="a_kafka_server:9092"                        \
+ *          --bootstrapServers="kafka_server_1:9092"                        \
  *          --topics="sample_tweets_json"                                   \
  *          --outputTopic="top_hashtags"
  * 
* - * TODO: Move this out this directory. */ public class TopHashtagsExample { @@ -149,7 +148,7 @@ public static void main(String args[]) { // The rest of the file implements DoFns to extract hashtags, formatting output, writing output // back to Kafka. Note that writing to Kafka is not a complete Dataflow Sink. It is a best-effort - // logging the results. + // logging of the results. private static final ObjectMapper jsonMapper = new ObjectMapper(); diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 90cf9746ef..7a429c11ff 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -52,7 +52,6 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.io.Serializable; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -133,15 +132,18 @@ public static class Builder { private Optional, Instant>> watermarkFn = Optional.absent(); private SerializableFunction, Consumer> - kafkaConsumerFactoryFn = - new SerializableFunction, Consumer>() { - public Consumer apply(Map config) { - return new KafkaConsumer<>(config); // default 0.9 consumer - } - }; + kafkaConsumerFactoryFn = kafka9Consumer; private Map mutableConsumerConfig = Maps.newHashMap(); + // default Kafka 0.9 Consumer supplier + private static SerializableFunction, Consumer> + kafka9Consumer = new SerializableFunction, Consumer>() { + public Consumer apply(Map config) { + return new KafkaConsumer<>(config); // default 0.9 consumer + } + }; + /** * set of properties that are not required or don't make sense for our consumer. */ @@ -249,8 +251,7 @@ public Builder withTimestampFn( } /** - * A function to calculate watermark. When this is not set, last record timestamp is returned - * in {@link UnboundedReader#getWatermark()}. + * A function to calculate watermark. Default is {@link UnboundedSource#getCurrentTimestamp()}. * * @param watermarkFn to calculate watermark at a record. * @return Builder @@ -272,7 +273,7 @@ public Builder withKafkaConsumerFactoryFn( } /** - * Build Unbounded Kafka Source + * Build Unbounded Kafka Source. * * @return UnboundedKafkaSource */ @@ -521,8 +522,8 @@ private void readNextBatch(boolean isFirstFetch) { // Use a longer timeout for first fetch. Kafka consumer seems to do better with poll() with // longer timeout initially. Looks like it does not handle initial connection setup properly - // with short polls and backoff policy in Dataflow might be making things worse for - // this case. In my tests it took ~5 seconds before first record was read with this + // with short polls (may also be affected by backoff policy in Dataflow). + // In my tests it took ~5 seconds before first record was read with this // hack and 20-30 seconds with out. long timeoutMillis = isFirstFetch ? 4000 : 100; ConsumerRecords records = consumer.poll(timeoutMillis); @@ -713,8 +714,7 @@ public ValueSourceBuilder withTimestampFn(SerializableFunction ti } /** - * A function to calculate watermark. When this is not set, last record timestamp is returned - * in {@link UnboundedReader#getWatermark()}. + * A function to calculate watermark. Default is {@link UnboundedSource#getCurrentTimestamp()}. * * @param watermarkFn to calculate watermark at a record. * @return Builder diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java index c54cf63e0b..36c951a17d 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java @@ -37,6 +37,7 @@ import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.util.CoderUtils; +import com.google.cloud.dataflow.sdk.util.SerializableUtils; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.common.collect.ImmutableList; @@ -198,9 +199,12 @@ public static void addCountingAsserts(PCollection input, long numElements) public void testUnboundedSource() { Pipeline p = TestPipeline.create(); int numElements = 1000; + UnboundedSource source = mkKafkaSource(numElements, new ValueAsTimestampFn()); + + SerializableUtils.ensureSerializable(source); PCollection input = p.apply(Read - .from(mkKafkaSource(numElements, null)) + .from(source) .withMaxNumRecords(numElements)); addCountingAsserts(input, numElements); @@ -271,7 +275,7 @@ public Instant apply(Long input) { @Test public void testUnboundedSourceCheckpointMark() throws Exception { - int numElements = 85; // make sure some partitions have more records than other + int numElements = 85; // 85 to make sure some partitions have more records than other. // create a single split: UnboundedSource source = @@ -285,13 +289,13 @@ public void testUnboundedSourceCheckpointMark() throws Exception { assertTrue(reader.start()); // Advance the source numToSkip-1 elements and manually save state. - for (long l = 0; l < numToSkip-1; ++l) { + for (long l = 0; l < numToSkip - 1; ++l) { reader.advance(); } // Confirm that we get the expected element in sequence before checkpointing. - assertEquals(numToSkip-1, (long) reader.getCurrent()); - assertEquals(numToSkip-1, reader.getCurrentTimestamp().getMillis()); + assertEquals(numToSkip - 1, (long) reader.getCurrent()); + assertEquals(numToSkip - 1, reader.getCurrentTimestamp().getMillis()); // Checkpoint and restart, and confirm that the source continues correctly. KafkaCheckpointMark mark = CoderUtils.clone( From f7746b8475090e75227414609db90b6a72c137c5 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 4 Mar 2016 10:34:32 -0800 Subject: [PATCH 41/72] minor --- contrib/examples/kafka/pom.xml | 11 +++++++ .../dataflow/contrib/kafka/KafkaSource.java | 30 +++++++++++-------- 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/contrib/examples/kafka/pom.xml b/contrib/examples/kafka/pom.xml index f5f39e3f27..3355020b62 100644 --- a/contrib/examples/kafka/pom.xml +++ b/contrib/examples/kafka/pom.xml @@ -174,5 +174,16 @@ google-cloud-dataflow-java-contrib-kafka ${project.version} + + org.slf4j + slf4j-api + 1.7.7 + + + org.slf4j + slf4j-jdk14 + 1.7.7 + runtime + diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java index 7a429c11ff..f3b98198a3 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java @@ -132,17 +132,17 @@ public static class Builder { private Optional, Instant>> watermarkFn = Optional.absent(); private SerializableFunction, Consumer> - kafkaConsumerFactoryFn = kafka9Consumer; - + kafkaConsumerFactoryFn = kafka9ConsumerFactory; private Map mutableConsumerConfig = Maps.newHashMap(); - // default Kafka 0.9 Consumer supplier + // default Kafka 0.9 Consumer supplier. static variable to avoid capturing 'this' private static SerializableFunction, Consumer> - kafka9Consumer = new SerializableFunction, Consumer>() { - public Consumer apply(Map config) { - return new KafkaConsumer<>(config); // default 0.9 consumer - } - }; + kafka9ConsumerFactory = + new SerializableFunction, Consumer>() { + public Consumer apply(Map config) { + return new KafkaConsumer<>(config); + } + }; /** * set of properties that are not required or don't make sense for our consumer. @@ -347,22 +347,18 @@ public UnboundedKafkaSource( public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { - Consumer consumer = kafkaConsumerFactoryFn.apply(consumerConfig); - List partitions = Lists.newArrayList(); // fetch partitions for each topic // sort by // round-robin assign the partitions to splits - try { + try (Consumer consumer = kafkaConsumerFactoryFn.apply(consumerConfig)) { for (String topic : topics) { for (PartitionInfo p : consumer.partitionsFor(topic)) { partitions.add(new TopicPartition(p.topic(), p.partition())); } } - } finally { - consumer.close(); } Collections.sort(partitions, new Comparator() { @@ -668,6 +664,14 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { return curTimestamp; } + + @Override + public long getSplitBacklogBytes() { + // TODO: we should do this. currently looks like we need to pass a partition and then + // seekToEnd() to find the latest offset. We could do that if that is useful. + return super.getSplitBacklogBytes(); + } + @Override public void close() throws IOException { Closeables.closeQuietly(consumer); From c9d4a69d8e0e8def3d3c7b7e542de35cdaef5baa Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 14:34:09 -0800 Subject: [PATCH 42/72] KafkaSource -> KafkaIO refactor --- .../kafka/examples/TopHashtagsExample.java | 6 +- .../kafka/{KafkaSource.java => KafkaIO.java} | 695 ++++++++---------- .../dataflow/contrib/kafka/KafkaRecord.java | 4 +- ...{KafkaSourceTest.java => KafkaIOTest.java} | 60 +- 4 files changed, 352 insertions(+), 413 deletions(-) rename contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/{KafkaSource.java => KafkaIO.java} (52%) rename contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/{KafkaSourceTest.java => KafkaIOTest.java} (85%) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 99a29e1d83..c69500dc46 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -16,7 +16,7 @@ package com.google.cloud.dataflow.contrib.kafka.examples; -import com.google.cloud.dataflow.contrib.kafka.KafkaSource; +import com.google.cloud.dataflow.contrib.kafka.KafkaIO; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.io.Read; @@ -58,7 +58,7 @@ import java.util.Map; /** - * This Dataflow app show cases {@link KafkaSource}. The application reads from a Kafka topic + * This Dataflow app show cases {@link KafkaIO}. The application reads from a Kafka topic * containing JSON Tweets, calculates top * hashtags in 10 minute window. The results are written back to a Kafka topic. * @@ -123,7 +123,7 @@ public static void main(String args[]) { final int windowSize = options.getSlidingWindowSize(); final int windowPeriod = options.getSlidingWindowPeriod(); - UnboundedSource kafkaSource = KafkaSource + UnboundedSource kafkaSource = KafkaIO .unboundedValueSourceBuilder() .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java similarity index 52% rename from contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java rename to contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index f3b98198a3..56889fbdb5 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaSource.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -16,21 +16,32 @@ package com.google.cloud.dataflow.contrib.kafka; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.api.client.repackaged.com.google.common.annotations.VisibleForTesting; import com.google.api.client.util.Maps; import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; +import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; +import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; +import com.google.cloud.dataflow.sdk.io.Read; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.PTransform; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.util.ExposedByteArrayInputStream; +import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PCollection; +import com.google.cloud.dataflow.sdk.values.PInput; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; -import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ComparisonChain; import com.google.common.collect.ImmutableList; @@ -47,22 +58,28 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.Collection; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.NoSuchElementException; import javax.annotation.Nullable; +// {@link #withConsumerProperty} or something like that ? +// {@link Coder} +// generally, feel free to drop @param and @return tags -- relatively unused in google style. +// probably worth javadoccing the splitting behavior extensively here, + /** * Dataflow Source for consuming Kafka sources. * @@ -82,9 +99,9 @@ * .... * */ -public class KafkaSource { +public class KafkaIO { - private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); + private static final Logger LOG = LoggerFactory.getLogger(KafkaIO.class); private static class NowTimestampFn implements SerializableFunction { @Override @@ -93,56 +110,43 @@ public Instant apply(T input) { } } - public static Builder unboundedSourceBuilder() { - return new Builder(); - } - - public static Builder unboundedByteSourceBuilder() { - return new Builder() - .withKeyCoder(ByteArrayCoder.of()) - .withValueCoder(ByteArrayCoder.of()); + private static class IdentityFn implements SerializableFunction { + @Override + public T apply(T input) { + return input; + } } /** - * Similar to {@link #unboundedSourceBuilder()}, where user in only interested in value, and - * wants to discard Kafak record key and metadata. - * - * @param value type - * @return {@link ValueSourceBuilder} + * XXX + * @return */ - public static ValueSourceBuilder unboundedValueSourceBuilder() { - return new ValueSourceBuilder( - new Builder() - .withKeyCoder(ByteArrayCoder.of())); + public static Reader reader() { + return new Reader( + new ArrayList(), + new ArrayList(), + ByteArrayCoder.of(), + ByteArrayCoder.of(), + null, + null, + Reader.kafka9ConsumerFactory, + Reader.defaultConsumerProperties, + Long.MAX_VALUE, + null); } - /** - * Builds Unbounded Kafka Source. - * - * @param K key type - * @param V value type - */ - public static class Builder { - - private List topics; - // future: let users specify subset of partitions to read. - private Coder keyCoder; - private Coder valueCoder; - private SerializableFunction, Instant> timestampFn = new NowTimestampFn<>(); - private Optional, Instant>> watermarkFn = - Optional.absent(); - private SerializableFunction, Consumer> - kafkaConsumerFactoryFn = kafka9ConsumerFactory; - private Map mutableConsumerConfig = Maps.newHashMap(); + public static class Reader extends PTransform>> { - // default Kafka 0.9 Consumer supplier. static variable to avoid capturing 'this' - private static SerializableFunction, Consumer> - kafka9ConsumerFactory = - new SerializableFunction, Consumer>() { - public Consumer apply(Map config) { - return new KafkaConsumer<>(config); - } - }; + private final List topics; + private final List topicPartitions; // mutually exclusive with topics + private final Coder keyCoder; + private final Coder valueCoder; + @Nullable private final SerializableFunction, Instant> timestampFn; + @Nullable private final SerializableFunction, Instant> watermarkFn; + private final SerializableFunction, Consumer> consumerFactoryFn; + private final Map consumerConfig; + private final long maxNumRecords; // bounded read, mainly for testing + private final Duration maxReadTime; // bounded read, mainly for testing /** * set of properties that are not required or don't make sense for our consumer. @@ -150,213 +154,314 @@ public Consumer apply(Map config) { private static final Map ignoredConsumerProperties = ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDecoderFn instead" - // "group.id", "enable.auto.commit", "auto.commit.interval.ms" : // lets allow these, applications can have better resume point for restarts. ); - private Builder() { - // set config defaults - mutableConsumerConfig.putAll(ImmutableMap.of( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - // default to latest offset when we are not resuming. - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", - // disable auto commit of offsets. we don't require group_id. could be enabled by user. - ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false)); + // set config defaults + private static final Map defaultConsumerProperties = + ImmutableMap.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + // default to latest offset when we are not resuming. + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", + // disable auto commit of offsets. we don't require group_id. could be enabled by user. + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + // default Kafka 0.9 Consumer supplier. static variable to avoid capturing 'this' + private static SerializableFunction, Consumer> + kafka9ConsumerFactory = + new SerializableFunction, Consumer>() { + public Consumer apply(Map config) { + return new KafkaConsumer<>(config); + } + }; + + public Reader( + List topics, + List topicPartitions, + Coder keyCoder, + Coder valueCoder, + @Nullable SerializableFunction, Instant> timestampFn, + @Nullable SerializableFunction, Instant> watermarkFn, + SerializableFunction, Consumer> consumerFactoryFn, + Map consumerConfig, + long maxNumRecords, + @Nullable Duration maxReadTime) { + + this.topics = topics; + this.topicPartitions = topicPartitions; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + this.timestampFn = timestampFn; + this.watermarkFn = watermarkFn; + this.consumerFactoryFn = consumerFactoryFn; + this.consumerConfig = consumerConfig; + this.maxNumRecords = maxNumRecords; + this.maxReadTime = maxReadTime; } - /** - * Set Kafka bootstrap servers (alternately, set "bootstrap.servers" Consumer property). - * - * @param bootstrapServers Bootstrap servers for Kafka. - * @return Builder - */ - public Builder withBootstrapServers(String bootstrapServers) { - return withConsumerProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + Reader withBootstrapServers(String bootstrapServers) { + return updateConsumerProperties( + ImmutableMap.of( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)); } - /** - * Set Kafka topics to be consumed. This is required. - * - * @param topics topics to read from - * @return Builder - */ - public Builder withTopics(Collection topics) { - this.topics = ImmutableList.copyOf(topics); - return this; + public Reader withTopics(List topics) { + return new Reader(ImmutableList.copyOf(topics), topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + } + + public Reader withTopicPartitions(List topicPartitions) { + return new Reader(topics, ImmutableList.copyOf(topicPartitions), keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** - * Set a {@link ConsumerConfig} configuration property. - * - * @param configKey configuration property name - * @param configValue value for configuration property - * @return Builder + * XXX timstampFns after this. */ - public Builder withConsumerProperty(String configKey, Object configValue) { - Preconditions.checkArgument(!ignoredConsumerProperties.containsKey(configKey), - "No need to configure '%s'. %s", configKey, ignoredConsumerProperties.get(configKey)); - mutableConsumerConfig.put(configKey, configValue); - return this; + public Reader withKeyCoder(Coder keyCoder) { + checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); + checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); + return new Reader(topics, topicPartitions, keyCoder, valueCoder, null, null, + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** - * Update consumer config properties. Note that this does not not discard already configured. - * Same as invoking #withConsumerProperty() with each entry. - * - * @param configUpdate updates to {@link ConsumerConfig} - * @return Builder + * XXX timstampFns after this. */ - public Builder withConsumerProperties(Map configUpdate) { - for (Entry e : configUpdate.entrySet()) { - withConsumerProperty(e.getKey(), e.getValue()); + public Reader withValueCoder(Coder valueCoder) { + checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); + checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); + return new Reader(topics, topicPartitions, keyCoder, valueCoder, null, null, + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + } + + public Reader withTimestampFn(SerializableFunction, Instant> timestampFn) { + checkNotNull(timestampFn); + return new Reader(topics, topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + } + + public Reader withWatermarkFn(SerializableFunction, Instant> watermarkFn) { + checkNotNull(watermarkFn); + return new Reader(topics, topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + } + + public Reader withConsumerFactoryFn( + SerializableFunction, Consumer> consumerFactoryFn) { + return new Reader(topics, topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + } + + public Reader updateConsumerProperties(Map configUpdates) { + for (String key : configUpdates.keySet()) { + checkArgument(!ignoredConsumerProperties.containsKey(key), + "No need to configure '%s'. %s", key, ignoredConsumerProperties.get(key)); } - return this; + + Map config = new HashMap<>(consumerConfig); + config.putAll(configUpdates); + + return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + watermarkFn, consumerFactoryFn, config, maxNumRecords, maxReadTime); } /** - * Set Coder for Key. - * - * @param keyCoder Coder for Key - * @return Builder + * Similar to {@link Read.Unbounded#withMaxNumRecords(long)}. Mainly used for tests and demo + * applications. */ - public Builder withKeyCoder(Coder keyCoder) { - this.keyCoder = keyCoder; - return this; + public Reader withMaxNumRecords(long maxNumRecords) { + return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, null); } /** - * Set Coder for Value. - * - * @param valueCoder Coder for Value - * @return Builder + * Similar to {@link Read.Unbounded#withMaxReadTime(Duration)}. Mainly used for tests and demo + * applications. */ - public Builder withValueCoder(Coder valueCoder) { - this.valueCoder = valueCoder; - return this; + public Reader withMaxReadTime(Duration maxReadTime) { + return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + watermarkFn, consumerFactoryFn, consumerConfig, Long.MAX_VALUE, maxReadTime); } /** - * A function to assign a timestamp to a record. When this is not set, processing timestamp - * (when record is processed by {@link UnboundedReader#advance()}) is used. - * - * @param timestampFn Function to calculate timestamp of a record. - * @return Builder + * XXX NOTE about doing this after setting other properties. + * @return */ - public Builder withTimestampFn( - SerializableFunction, Instant> timestampFn) { - this.timestampFn = timestampFn; - return this; + public ReaderWithMetadata withMetadata() { + return new ReaderWithMetadata(this, + timestampFn != null ? unwrapKafkaAndThen(timestampFn) : null, + watermarkFn != null ? unwrapKafkaAndThen(watermarkFn) : null); } - /** - * A function to calculate watermark. Default is {@link UnboundedSource#getCurrentTimestamp()}. - * - * @param watermarkFn to calculate watermark at a record. - * @return Builder - */ - public Builder withWatermarkFn( - SerializableFunction, Instant> watermarkFn) { - this.watermarkFn = Optional.of(watermarkFn); - return this; + private static SerializableFunction, OutT> + unwrapKafkaAndThen(final SerializableFunction, OutT> fn) { + return new SerializableFunction, OutT>() { + public OutT apply(KafkaRecord record) { + return fn.apply(KV.of(record.getKey(), record.getValue())); + } + }; + } + + @VisibleForTesting + public UnboundedKafkaSource> makeSource() { + return new UnboundedKafkaSource>( + -1, + topics, + topicPartitions, + keyCoder, + valueCoder, + KvCoder.of(keyCoder, valueCoder), + unwrapKafkaAndThen(new IdentityFn>()), + timestampFn == null ? null : unwrapKafkaAndThen(timestampFn), + Optional.fromNullable(watermarkFn == null ? null : unwrapKafkaAndThen(watermarkFn)), + consumerFactoryFn, + consumerConfig); } - /** - * A factory to create Kafka {@link Consumer} from consumer configuration. - * Mainly used for tests. - */ - public Builder withKafkaConsumerFactoryFn( - SerializableFunction, Consumer> kafkaConsumerFactoryFn) { - this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn; - return this; + @Override + public PCollection> apply(PInput input) { + return applyHelper(input, makeSource()); } /** - * Build Unbounded Kafka Source. - * - * @return UnboundedKafkaSource + * Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. */ - public UnboundedSource, KafkaCheckpointMark> build() { + PCollection applyHelper(PInput input, UnboundedSource source) { + Read.Unbounded unbounded = Read.from(source); + PTransform> transform = unbounded; + + if (maxNumRecords < Long.MAX_VALUE) { + transform = unbounded.withMaxNumRecords(maxNumRecords); + } else if (maxReadTime != null) { + transform = unbounded.withMaxReadTime(maxReadTime); + } + + return input.getPipeline().apply(transform); + } + } - ImmutableMap consumerConfig = ImmutableMap.copyOf(mutableConsumerConfig); + public static class ReaderWithMetadata + extends PTransform>> { - Preconditions.checkNotNull( - consumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), - "Kafka bootstrap servers should be set"); - Preconditions.checkNotNull(topics, "Kafka topics should be set"); - Preconditions.checkArgument(!topics.isEmpty(), "At least one topic is required"); - Preconditions.checkNotNull(keyCoder, "Coder for Kafka key bytes is required"); - Preconditions.checkNotNull(valueCoder, "Coder for Kafka values bytes is required"); + private final Reader kvReader; + @Nullable private final SerializableFunction, Instant> timestampFn; + @Nullable private final SerializableFunction, Instant> watermarkFn; + + ReaderWithMetadata( + Reader kvReader, + @Nullable SerializableFunction, Instant> timestampFn, + @Nullable SerializableFunction, Instant> watermarkFn) { - return new UnboundedKafkaSource( + this.kvReader = kvReader; + this.timestampFn = timestampFn; + this.watermarkFn = watermarkFn; + } + + // Interface Note: + // Instead of repeating many of the builder methods ('withTopics()' etc) in Reader, we expect + // the user to set all those before Reader.withMetaData(). we still need to let users + // override timestamp functions (in cases where these functions need metadata). + + public ReaderWithMetadata withTimestampFn( + SerializableFunction, Instant> timestampFn) { + checkNotNull(timestampFn); + return new ReaderWithMetadata(kvReader, timestampFn, watermarkFn); + } + + public ReaderWithMetadata withWatermarkFn( + SerializableFunction, Instant> watermarkFn) { + checkNotNull(watermarkFn); + return new ReaderWithMetadata(kvReader, timestampFn, watermarkFn); + } + + @VisibleForTesting + public UnboundedKafkaSource> makeSource() { + return new UnboundedKafkaSource<>( -1, - consumerConfig, - topics, - keyCoder, - valueCoder, + kvReader.topics, + kvReader.topicPartitions, + kvReader.keyCoder, + kvReader.valueCoder, + KafkaRecordCoder.of(kvReader.keyCoder, kvReader.valueCoder), + new IdentityFn>(), timestampFn, - watermarkFn, - kafkaConsumerFactoryFn, - ImmutableList.of() // no assigned partitions yet. - ); + Optional.fromNullable(watermarkFn), + kvReader.consumerFactoryFn, + kvReader.consumerConfig); + } + + @Override + public PCollection> apply(PInput input) { + return kvReader.applyHelper(input, makeSource()); } } /** Static class, prevent instantiation. */ - private KafkaSource() {} + private KafkaIO() {} - private static class UnboundedKafkaSource - extends UnboundedSource, KafkaCheckpointMark> { + private static class UnboundedKafkaSource + extends UnboundedSource { private final int id; // split id, mainly for debugging - private final ImmutableMap consumerConfig; private final List topics; + private final List assignedPartitions; private final Coder keyCoder; private final Coder valueCoder; + private final Coder defaultOutputCoder; + private final SerializableFunction, T> converterFn; // covert to userTuype. private final SerializableFunction, Instant> timestampFn; // would it be a good idea to pass currentTimestamp to watermarkFn? private final Optional, Instant>> watermarkFn; private - SerializableFunction, Consumer> kafkaConsumerFactoryFn; - private final List assignedPartitions; + SerializableFunction, Consumer> consumerFactoryFn; + private final Map consumerConfig; public UnboundedKafkaSource( int id, - ImmutableMap consumerConfig, List topics, + List assignedPartitions, Coder keyCoder, Coder valueCoder, - SerializableFunction, Instant> timestampFn, + Coder defaultOutputCoder, + SerializableFunction, T> converterFn, + @Nullable SerializableFunction, Instant> timestampFn, Optional, Instant>> watermarkFn, - SerializableFunction, Consumer> kafkaConsumerFactoryFn, - List assignedPartitions) { + SerializableFunction, Consumer> consumerFactoryFn, + Map consumerConfig) { this.id = id; - this.consumerConfig = consumerConfig; + this.assignedPartitions = assignedPartitions; this.topics = topics; this.keyCoder = keyCoder; this.valueCoder = valueCoder; - this.timestampFn = timestampFn; + this.defaultOutputCoder = defaultOutputCoder; + this.converterFn = converterFn; + this.timestampFn = + (timestampFn == null ? new NowTimestampFn>() : timestampFn); this.watermarkFn = watermarkFn; - this.kafkaConsumerFactoryFn = kafkaConsumerFactoryFn; - this.assignedPartitions = ImmutableList.copyOf(assignedPartitions); + this.consumerFactoryFn = consumerFactoryFn; + this.consumerConfig = consumerConfig; } @Override - public List> generateInitialSplits( + public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { - List partitions = Lists.newArrayList(); + List partitions = new ArrayList<>(assignedPartitions); - // fetch partitions for each topic - // sort by - // round-robin assign the partitions to splits + // (a) fetch partitions for each topic + // (b) sort by + // (c) round-robin assign the partitions to splits - try (Consumer consumer = kafkaConsumerFactoryFn.apply(consumerConfig)) { - for (String topic : topics) { - for (PartitionInfo p : consumer.partitionsFor(topic)) { - partitions.add(new TopicPartition(p.topic(), p.partition())); + if (partitions.isEmpty()) { + try (Consumer consumer = consumerFactoryFn.apply(consumerConfig)) { + for (String topic : topics) { + for (PartitionInfo p : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(p.topic(), p.partition())); + } } } } @@ -371,8 +476,8 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { } }); - Preconditions.checkArgument(desiredNumSplits > 0); - Preconditions.checkState(partitions.size() > 0, + checkArgument(desiredNumSplits > 0); + checkState(partitions.size() > 0, "Could not find any partitions. Please check Kafka configuration and topic names"); int numSplits = Math.min(desiredNumSplits, partitions.size()); @@ -387,7 +492,7 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { assignments.get(i % numSplits).add(partitions.get(i)); } - List> result = Lists.newArrayList(); + List> result = Lists.newArrayList(); for (int i = 0; i < numSplits; i++) { List assignedToSplit = assignments.get(i); @@ -395,35 +500,37 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { LOG.info("Partitions assigned to split {} (total {}): {}", i, assignedToSplit.size(), Joiner.on(",").join(assignedToSplit)); - result.add(new UnboundedKafkaSource( + result.add(new UnboundedKafkaSource( i, - this.consumerConfig, this.topics, + assignedToSplit, this.keyCoder, this.valueCoder, + this.defaultOutputCoder, + this.converterFn, this.timestampFn, this.watermarkFn, - this.kafkaConsumerFactoryFn, - assignedToSplit)); + this.consumerFactoryFn, + this.consumerConfig)); } return result; } @Override - public UnboundedKafkaReader createReader(PipelineOptions options, + public UnboundedKafkaReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { if (assignedPartitions.isEmpty()) { LOG.warn("hack: working around DirectRunner issue. It does not generateSplits()"); // generate single split and return reader from it. try { - return new UnboundedKafkaReader( + return new UnboundedKafkaReader( generateInitialSplits(1, options).get(0), checkpointMark); } catch (Exception e) { Throwables.propagate(e); } } - return new UnboundedKafkaReader(this, checkpointMark); + return new UnboundedKafkaReader(this, checkpointMark); } @Override @@ -438,17 +545,19 @@ public boolean requiresDeduping() { @Override public void validate() { - // TODO anything to do here? + checkNotNull(consumerConfig.get(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG), + "Kafka bootstrap servers should be set"); + checkArgument(topics.size() > 0 || assignedPartitions.size() > 0, + "Kafka topics or topic_partitions are required"); } @Override - public Coder> getDefaultOutputCoder() { - return KafkaRecordCoder.of(keyCoder, valueCoder); + public Coder getDefaultOutputCoder() { + return defaultOutputCoder; } } - private static class UnboundedKafkaReader - extends UnboundedReader> { + private static class UnboundedKafkaReader extends UnboundedReader { // maintains state of each assigned partition (buffered records and consumed offset) private static class PartitionState { @@ -462,7 +571,7 @@ private static class PartitionState { } } - private final UnboundedKafkaSource source; + private final UnboundedKafkaSource source; private final String name; private Consumer consumer; private final List partitionStates; @@ -478,7 +587,7 @@ public String toString() { } public UnboundedKafkaReader( - UnboundedKafkaSource source, + UnboundedKafkaSource source, @Nullable KafkaCheckpointMark checkpointMark) { this.source = source; @@ -495,8 +604,7 @@ public PartitionState apply(TopicPartition tp) { // a) verify that assigned and check-pointed partitions match exactly // b) set consumed offsets - Preconditions.checkState( - checkpointMark.getPartitions().size() == source.assignedPartitions.size(), + checkState(checkpointMark.getPartitions().size() == source.assignedPartitions.size(), "checkPointMark and assignedPartitions should match"); // we could consider allowing a mismatch, though it is not expected in current Dataflow @@ -504,7 +612,7 @@ public PartitionState apply(TopicPartition tp) { PartitionMark ckptMark = checkpointMark.getPartitions().get(i); TopicPartition assigned = source.assignedPartitions.get(i); - Preconditions.checkState(ckptMark.getTopicPartition().equals(assigned), + checkState(ckptMark.getTopicPartition().equals(assigned), "checkpointed partition %s and assinged partition %s don't match at position %d", ckptMark.getTopicPartition(), assigned, i); @@ -539,7 +647,7 @@ private void readNextBatch(boolean isFirstFetch) { @Override public boolean start() throws IOException { - consumer = source.kafkaConsumerFactoryFn.apply(source.consumerConfig); + consumer = source.consumerFactoryFn.apply(source.consumerConfig); consumer.assign(source.assignedPartitions); // seek to consumedOffset + 1 if it is set @@ -649,14 +757,14 @@ public PartitionMark apply(PartitionState p) { } @Override - public UnboundedSource, ?> getCurrentSource() { + public UnboundedSource getCurrentSource() { return source; } @Override - public KafkaRecord getCurrent() throws NoSuchElementException { + public T getCurrent() throws NoSuchElementException { // should we delay updating consumed offset till this point? Mostly not required. - return curRecord; + return source.converterFn.apply(curRecord); } @Override @@ -667,8 +775,9 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { @Override public long getSplitBacklogBytes() { - // TODO: we should do this. currently looks like we need to pass a partition and then - // seekToEnd() to find the latest offset. We could do that if that is useful. + // TODO: fetch latest offsets to estimate backlog. currently looks like we need to pause a + // partition and then seekToEnd() to find the latest offset. + // Hopefully Kafka consumer supports fetching this cleanly. return super.getSplitBacklogBytes(); } @@ -677,180 +786,4 @@ public void close() throws IOException { Closeables.closeQuietly(consumer); } } - - // Builder, Source, and Reader wrappers. Often user is only interested in Value in KafkaRecord : - - /** - * Builder for Kafka Source where user is not interested in Kafka metadata and key for a record, - * but just the value. - */ - public static class ValueSourceBuilder { - - private Builder underlying; - - private ValueSourceBuilder(Builder underlying) { - this.underlying = underlying; - } - - public ValueSourceBuilder withBootstrapServers(String bootstrapServers) { - return new ValueSourceBuilder(underlying.withBootstrapServers(bootstrapServers)); - } - - public ValueSourceBuilder withTopics(Collection topics) { - return new ValueSourceBuilder(underlying.withTopics(topics)); - } - - public ValueSourceBuilder withConsumerProperty(String configKey, Object configValue) { - return new ValueSourceBuilder(underlying.withConsumerProperty(configKey, configValue)); - } - - public ValueSourceBuilder withConsumerProperties(Map configToUpdate) { - return new ValueSourceBuilder(underlying.withConsumerProperties(configToUpdate)); - } - - public ValueSourceBuilder withValueCoder(Coder valueCoder) { - return new ValueSourceBuilder(underlying.withValueCoder(valueCoder)); - } - - public ValueSourceBuilder withTimestampFn(SerializableFunction timestampFn) { - return new ValueSourceBuilder( - underlying.withTimestampFn(unwrapKafkaAndThen(timestampFn))); - } - - /** - * A function to calculate watermark. Default is {@link UnboundedSource#getCurrentTimestamp()}. - * - * @param watermarkFn to calculate watermark at a record. - * @return Builder - */ - public ValueSourceBuilder withWatermarkFn(SerializableFunction watermarkFn) { - return new ValueSourceBuilder( - underlying.withTimestampFn(unwrapKafkaAndThen(watermarkFn))); - } - - /** - * A factory to create Kafka {@link Consumer} from consumer configuration. - * Mainly used for tests. - * @param kafkaConsumerFactoryFn function to create - * @return - */ - public ValueSourceBuilder withKafkaConsumerFactoryFn( - SerializableFunction, Consumer> kafkaConsumerFactoryFn) { - return new ValueSourceBuilder( - underlying.withKafkaConsumerFactoryFn(kafkaConsumerFactoryFn)); - } - - public UnboundedSource build() { - return new UnboundedKafkaValueSource((UnboundedKafkaSource) underlying.build()); - } - - private static - SerializableFunction, OutT> unwrapKafkaAndThen( - final SerializableFunction fn) { - return new SerializableFunction, OutT>() { - public OutT apply(KafkaRecord record) { - return fn.apply(record.getValue()); - } - }; - } - } - - /** - * Usually the users are only interested in value in KafkaRecord. This is a convenient class - * to strip out other fields in KafkaRecord returned by UnboundedKafkaValueSource - */ - private static class UnboundedKafkaValueSource - extends UnboundedSource { - - private final UnboundedKafkaSource underlying; - - public UnboundedKafkaValueSource(UnboundedKafkaSource underlying) { - this.underlying = underlying; - } - - @Override - public List> generateInitialSplits( - int desiredNumSplits, PipelineOptions options) throws Exception { - return Lists.transform(underlying.generateInitialSplits(desiredNumSplits, options), - new Function, UnboundedKafkaValueSource>() { - public UnboundedKafkaValueSource apply(UnboundedKafkaSource input) { - return new UnboundedKafkaValueSource(input); - } - }); - } - - @Override - public UnboundedReader createReader( - PipelineOptions options, KafkaCheckpointMark checkpointMark) { - return new UnboundedKafkaValueReader(this, - underlying.createReader(options, checkpointMark)); - } - - @Override - public Coder getCheckpointMarkCoder() { - return underlying.getCheckpointMarkCoder(); - } - - @Override - public void validate() { - underlying.validate(); - } - - @Override - public Coder getDefaultOutputCoder() { - return underlying.valueCoder; - } - } - - private static class UnboundedKafkaValueReader extends UnboundedReader { - - private final UnboundedKafkaValueSource source; - private final UnboundedKafkaReader underlying; - - public UnboundedKafkaValueReader(UnboundedKafkaValueSource source, - UnboundedKafkaReader underlying) { - this.source = source; - this.underlying = underlying; - } - - @Override - public boolean start() throws IOException { - return underlying.start(); - } - - @Override - public boolean advance() throws IOException { - return underlying.advance(); - } - - @Override - public Instant getWatermark() { - return underlying.getWatermark(); - } - - @Override - public CheckpointMark getCheckpointMark() { - return underlying.getCheckpointMark(); - } - - @Override - public UnboundedKafkaValueSource getCurrentSource() { - return source; - } - - @Override - public V getCurrent() throws NoSuchElementException { - return underlying.getCurrent().getValue(); - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - return underlying.getCurrentTimestamp(); - } - - @Override - public void close() throws IOException { - underlying.close(); - } - } } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index 9ef8137858..ff40d84b3a 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -29,7 +29,7 @@ public class KafkaRecord implements Serializable { private final int partition; private final long offset; private final K key; - private final V value; + private final V value; // XXX TODO: use KV instead public KafkaRecord( String topic, @@ -73,7 +73,7 @@ public boolean equals(Object obj) { return topic.equals(other.topic) && partition == other.partition && offset == other.offset - && key.equals(other.key) + && key.equals(other.key) // XXX KV.equals() && value.equals(other.value); } else { return false; diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java similarity index 85% rename from contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java rename to contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 36c951a17d..0c4517968d 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaSourceTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Reader; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.io.Read; @@ -36,8 +37,9 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; +import com.google.cloud.dataflow.sdk.transforms.Values; import com.google.cloud.dataflow.sdk.util.CoderUtils; -import com.google.cloud.dataflow.sdk.util.SerializableUtils; +import com.google.cloud.dataflow.sdk.values.KV; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.common.collect.ImmutableList; @@ -67,13 +69,14 @@ * Tests of {@link KafkaSource}. */ @RunWith(JUnit4.class) -public class KafkaSourceTest { +public class KafkaIOTest { /* * The tests below borrow code and structure from CountingSourceTest. In addition verifies * the reader interleaves the records from multiple partitions. * * Other tests to consider : * - test KafkaRecordCoder + * - test with manual partitions */ // Update mock consumer with records distributed among the given topics, each with given number @@ -155,23 +158,23 @@ public Consumer apply(Map config) { * Creates a consumer with two topics, with 5 partitions each. * numElements are (round-robin) assigned all the 10 partitions. */ - private static UnboundedSource mkKafkaSource( + private static UnboundedSource, KafkaCheckpointMark> mkKafkaSource( int numElements, - @Nullable SerializableFunction timestampFn) { + @Nullable SerializableFunction, Instant> timestampFn) { List topics = ImmutableList.of("topic_a", "topic_b"); - KafkaSource.ValueSourceBuilder builder = KafkaSource.unboundedValueSourceBuilder() + Reader reader = KafkaIO.reader() .withBootstrapServers("none") .withTopics(topics) - .withKafkaConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions + .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions .withValueCoder(BigEndianLongCoder.of()); if (timestampFn != null) { - builder = builder.withTimestampFn(timestampFn); + reader = reader.withTimestampFn(timestampFn); } - return builder.build(); + return reader.makeSource(); } public static void addCountingAsserts(PCollection input, long numElements) { @@ -199,13 +202,12 @@ public static void addCountingAsserts(PCollection input, long numElements) public void testUnboundedSource() { Pipeline p = TestPipeline.create(); int numElements = 1000; - UnboundedSource source = mkKafkaSource(numElements, new ValueAsTimestampFn()); - SerializableUtils.ensureSerializable(source); - - PCollection input = p.apply(Read - .from(source) - .withMaxNumRecords(numElements)); + PCollection input = p + .apply(Read + .from(mkKafkaSource(numElements, new ValueAsTimestampFn())) + .withMaxNumRecords(numElements)) + .apply(Values.create()); addCountingAsserts(input, numElements); p.run(); @@ -224,9 +226,11 @@ public void testUnboundedSourceTimestamps() { Pipeline p = TestPipeline.create(); int numElements = 1000; - PCollection input = p.apply(Read - .from(mkKafkaSource(numElements, new ValueAsTimestampFn())) - .withMaxNumRecords(numElements)); + PCollection input = p + .apply(Read.from(mkKafkaSource(numElements, new ValueAsTimestampFn())) + .withMaxNumRecords(numElements)) + .apply(Values.create()); + addCountingAsserts(input, numElements); PCollection diffs = input @@ -245,8 +249,8 @@ public void testUnboundedSourceSplits() throws Exception { int numElements = 1000; int numSplits = 10; - UnboundedSource initial = mkKafkaSource(numElements, null); - List> splits = + UnboundedSource, ?> initial = mkKafkaSource(numElements, null); + List, ?>> splits = initial.generateInitialSplits(numSplits, p.getOptions()); assertEquals("Expected exact splitting", numSplits, splits.size()); @@ -255,7 +259,8 @@ public void testUnboundedSourceSplits() throws Exception { PCollectionList pcollections = PCollectionList.empty(p); for (int i = 0; i < splits.size(); ++i) { pcollections = pcollections.and( - p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit))); + p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit)) + .apply(Values.create())); } PCollection input = pcollections.apply(Flatten.pCollections()); @@ -266,10 +271,11 @@ public void testUnboundedSourceSplits() throws Exception { /** * A timestamp function that uses the given value as the timestamp. */ - private static class ValueAsTimestampFn implements SerializableFunction { + private static class ValueAsTimestampFn + implements SerializableFunction, Instant> { @Override - public Instant apply(Long input) { - return new Instant(input); + public Instant apply(KV input) { + return new Instant(input.getValue()); } } @@ -278,12 +284,12 @@ public void testUnboundedSourceCheckpointMark() throws Exception { int numElements = 85; // 85 to make sure some partitions have more records than other. // create a single split: - UnboundedSource source = + UnboundedSource, KafkaCheckpointMark> source = mkKafkaSource(numElements, new ValueAsTimestampFn()) .generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create()) .get(0); - UnboundedReader reader = source.createReader(null, null); + UnboundedReader> reader = source.createReader(null, null); final int numToSkip = 3; // advance once: assertTrue(reader.start()); @@ -294,7 +300,7 @@ public void testUnboundedSourceCheckpointMark() throws Exception { } // Confirm that we get the expected element in sequence before checkpointing. - assertEquals(numToSkip - 1, (long) reader.getCurrent()); + assertEquals(numToSkip - 1, (long) reader.getCurrent().getValue()); assertEquals(numToSkip - 1, reader.getCurrentTimestamp().getMillis()); // Checkpoint and restart, and confirm that the source continues correctly. @@ -306,7 +312,7 @@ public void testUnboundedSourceCheckpointMark() throws Exception { // Confirm that we get the next elements in sequence. // This also confirms that Reader interleaves records from each partitions by the reader. for (int i = numToSkip; i < numElements; i++) { - assertEquals(i, (long) reader.getCurrent()); + assertEquals(i, (long) reader.getCurrent().getValue()); assertEquals(i, reader.getCurrentTimestamp().getMillis()); reader.advance(); } From ec2e74e5e6814932873a67662dfd7ec25557af98 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 15:00:48 -0800 Subject: [PATCH 43/72] unit tests work --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 16 ++++++++-------- .../dataflow/contrib/kafka/KafkaIOTest.java | 2 +- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 56889fbdb5..c0097eb2e1 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -21,9 +21,7 @@ import static com.google.common.base.Preconditions.checkState; import com.google.api.client.repackaged.com.google.common.annotations.VisibleForTesting; -import com.google.api.client.util.Maps; import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; -import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; @@ -75,7 +73,6 @@ import javax.annotation.Nullable; -// {@link #withConsumerProperty} or something like that ? // {@link Coder} // generally, feel free to drop @param and @return tags -- relatively unused in google style. // probably worth javadoccing the splitting behavior extensively here, @@ -726,11 +723,14 @@ public boolean advance() throws IOException { } } + private static byte[] nullBytes = new byte[0]; private static T decode(byte[] bytes, Coder coder) throws IOException { - if (bytes == null) { - return null; // is this the right thing to do? - } - return coder.decode(new ExposedByteArrayInputStream(bytes), Coder.Context.OUTER); + // If 'bytes' is null use byte[0]. It is common for key in Kakfa record to be null. + // This makes it impossible for user to distinguish between zero length byte and null. + // Alternately, we could have a ByteArrayCoder that handles nulls, use that for default + // coder. + byte[] toDecode = bytes == null ? nullBytes : bytes; + return coder.decode(new ExposedByteArrayInputStream(toDecode), Coder.Context.OUTER); } @Override @@ -783,7 +783,7 @@ public long getSplitBacklogBytes() { @Override public void close() throws IOException { - Closeables.closeQuietly(consumer); + Closeables.close(consumer, true); } } } diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 0c4517968d..68438bfbdd 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -260,7 +260,7 @@ public void testUnboundedSourceSplits() throws Exception { for (int i = 0; i < splits.size(); ++i) { pcollections = pcollections.and( p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit)) - .apply(Values.create())); + .apply("collection " + i, Values.create())); } PCollection input = pcollections.apply(Flatten.pCollections()); From 79298c7e1eb6be0bdc25bf3bc04e25636c427342 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 15:34:11 -0800 Subject: [PATCH 44/72] add manual partitions test --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 4 ++ .../dataflow/contrib/kafka/KafkaIOTest.java | 48 ++++++++++++++++++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index c0097eb2e1..3975efe710 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -205,11 +205,15 @@ Reader withBootstrapServers(String bootstrapServers) { } public Reader withTopics(List topics) { + checkState(topicPartitions.isEmpty(), "Only topics or topicPartitions can be set, not both"); + return new Reader(ImmutableList.copyOf(topics), topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } public Reader withTopicPartitions(List topicPartitions) { + checkState(topics.isEmpty(), "Only topics or topicPartitions can be set, not both"); + return new Reader(topics, ImmutableList.copyOf(topicPartitions), keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 68438bfbdd..02eeaa3436 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -76,7 +76,6 @@ public class KafkaIOTest { * * Other tests to consider : * - test KafkaRecordCoder - * - test with manual partitions */ // Update mock consumer with records distributed among the given topics, each with given number @@ -177,6 +176,22 @@ private static UnboundedSource, KafkaCheckpointMark> mkKafkaSou return reader.makeSource(); } + private static class AssertMultipleOf implements SerializableFunction, Void> { + private final int num; + + public AssertMultipleOf(int num) { + this.num = num; + } + + @Override + public Void apply(Iterable values) { + for (Long v : values) { + assertEquals(0, v % num); + } + return null; + } + } + public static void addCountingAsserts(PCollection input, long numElements) { // Count == numElements DataflowAssert @@ -213,6 +228,37 @@ public void testUnboundedSource() { p.run(); } + @Test + @Category(RunnableOnService.class) + public void testUnboundedSourceWithExplicitPartitions() { + Pipeline p = TestPipeline.create(); + int numElements = 1000; + + List topics = ImmutableList.of("test"); + + Reader reader = KafkaIO.reader() + .withBootstrapServers("none") + .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5))) + .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 10 partitions + .withValueCoder(BigEndianLongCoder.of()) + .withMaxNumRecords(numElements/10); + + PCollection input = p + .apply(reader) + .apply(Values.create()); + + // assert that every element is a multiple of 5. + DataflowAssert + .that(input) + .satisfies(new AssertMultipleOf(5)); + + DataflowAssert + .thatSingleton(input.apply(Count.globally())) + .isEqualTo(numElements / 10L); + + p.run(); + } + private static class ElementValueDiff extends DoFn { @Override public void processElement(ProcessContext c) throws Exception { From 56411da5903724bee51c153b2ab98c8a27522e8e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 15:58:03 -0800 Subject: [PATCH 45/72] update example app --- .../kafka/examples/TopHashtagsExample.java | 21 +++++++++---------- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index c69500dc46..ad846c453d 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -17,10 +17,9 @@ package com.google.cloud.dataflow.contrib.kafka.examples; import com.google.cloud.dataflow.contrib.kafka.KafkaIO; +import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Reader; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; -import com.google.cloud.dataflow.sdk.io.Read; -import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.options.Default; import com.google.cloud.dataflow.sdk.options.Description; import com.google.cloud.dataflow.sdk.options.PipelineOptions; @@ -31,6 +30,7 @@ import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.transforms.Top; +import com.google.cloud.dataflow.sdk.transforms.Values; import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow; import com.google.cloud.dataflow.sdk.transforms.windowing.SlidingWindows; import com.google.cloud.dataflow.sdk.transforms.windowing.Window; @@ -123,16 +123,15 @@ public static void main(String args[]) { final int windowSize = options.getSlidingWindowSize(); final int windowPeriod = options.getSlidingWindowPeriod(); - UnboundedSource kafkaSource = KafkaIO - .unboundedValueSourceBuilder() + Reader reader = KafkaIO.reader() .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) .withValueCoder(StringUtf8Coder.of()) - .withTimestampFn(timestampFn) - .build(); + .withTimestampFn(timestampFn); pipeline - .apply(Read.from(kafkaSource).named("sample_tweets")) + .apply("sample_tweets", reader) + .apply(Values.create()) .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows .of(Duration.standardMinutes(windowSize)) @@ -168,12 +167,12 @@ public void processElement(ProcessContext ctx) throws Exception { } // extract timestamp from "timestamp_ms" field. - private static SerializableFunction timestampFn = - new SerializableFunction() { + private static SerializableFunction, Instant> timestampFn = + new SerializableFunction, Instant>() { @Override - public Instant apply(String json) { + public Instant apply(KV kv) { try { - long tsMillis = jsonMapper.readTree(json).path("timestamp_ms").asLong(); + long tsMillis = jsonMapper.readTree(kv.getValue()).path("timestamp_ms").asLong(); return tsMillis == 0 ? Instant.now() : new Instant(tsMillis); } catch (Exception e) { throw Throwables.propagate(e); diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 3975efe710..8412476cef 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -198,7 +198,7 @@ public Reader( this.maxReadTime = maxReadTime; } - Reader withBootstrapServers(String bootstrapServers) { + public Reader withBootstrapServers(String bootstrapServers) { return updateConsumerProperties( ImmutableMap.of( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)); From 900d339bb05a968ab073dd88229f5de79c1f9c2c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 18:16:26 -0800 Subject: [PATCH 46/72] rename Reader to Read. more javadoc --- .../kafka/examples/TopHashtagsExample.java | 4 +- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 148 +++++++++++------- .../dataflow/contrib/kafka/KafkaIOTest.java | 6 +- 3 files changed, 93 insertions(+), 65 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index ad846c453d..441dc0a49b 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -17,7 +17,7 @@ package com.google.cloud.dataflow.contrib.kafka.examples; import com.google.cloud.dataflow.contrib.kafka.KafkaIO; -import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Reader; +import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Read; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.Default; @@ -123,7 +123,7 @@ public static void main(String args[]) { final int windowSize = options.getSlidingWindowSize(); final int windowPeriod = options.getSlidingWindowPeriod(); - Reader reader = KafkaIO.reader() + Read reader = KafkaIO.read() .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) .withValueCoder(StringUtf8Coder.of()) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 8412476cef..0084bc63ba 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -26,7 +26,7 @@ import com.google.cloud.dataflow.sdk.coders.Coder; import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; -import com.google.cloud.dataflow.sdk.io.Read; +import com.google.cloud.dataflow.sdk.io.Read.Unbounded; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; @@ -115,24 +115,31 @@ public T apply(T input) { } /** - * XXX - * @return + * Creates and uninitialized {@ Read} transform. Before use, basic Kafka configuration should set + * with {@link Read#withBootstrapServers(String)}, {@link Read#withTopics(List)}. + * Other optional settings include key and value coders, custom timestamp and watermark + * functions. Additionally {@link Read#withMetadata()} provides access to Kafka metadata for + * each record (topic name, partition, and offset). */ - public static Reader reader() { - return new Reader( + public static Read read() { + return new Read( new ArrayList(), new ArrayList(), ByteArrayCoder.of(), ByteArrayCoder.of(), null, null, - Reader.kafka9ConsumerFactory, - Reader.defaultConsumerProperties, + Read.kafka9ConsumerFactory, + Read.defaultConsumerProperties, Long.MAX_VALUE, null); } - public static class Reader extends PTransform>> { + /** + * A transform to read from Kafka topics. See {@link KafkaIO#read()} for more information on + * configuration. + */ + public static class Read extends PTransform>> { private final List topics; private final List topicPartitions; // mutually exclusive with topics @@ -140,7 +147,8 @@ public static class Reader extends PTransform private final Coder valueCoder; @Nullable private final SerializableFunction, Instant> timestampFn; @Nullable private final SerializableFunction, Instant> watermarkFn; - private final SerializableFunction, Consumer> consumerFactoryFn; + private final + SerializableFunction, Consumer> consumerFactoryFn; private final Map consumerConfig; private final long maxNumRecords; // bounded read, mainly for testing private final Duration maxReadTime; // bounded read, mainly for testing @@ -174,7 +182,7 @@ public Consumer apply(Map config) { } }; - public Reader( + public Read( List topics, List topicPartitions, Coder keyCoder, @@ -198,65 +206,85 @@ public Reader( this.maxReadTime = maxReadTime; } - public Reader withBootstrapServers(String bootstrapServers) { + public Read withBootstrapServers(String bootstrapServers) { return updateConsumerProperties( ImmutableMap.of( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)); } - public Reader withTopics(List topics) { + public Read withTopics(List topics) { checkState(topicPartitions.isEmpty(), "Only topics or topicPartitions can be set, not both"); - return new Reader(ImmutableList.copyOf(topics), topicPartitions, keyCoder, valueCoder, + return new Read(ImmutableList.copyOf(topics), topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - public Reader withTopicPartitions(List topicPartitions) { + public Read withTopicPartitions(List topicPartitions) { checkState(topics.isEmpty(), "Only topics or topicPartitions can be set, not both"); - return new Reader(topics, ImmutableList.copyOf(topicPartitions), keyCoder, valueCoder, + return new Read(topics, ImmutableList.copyOf(topicPartitions), keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** - * XXX timstampFns after this. + * Set {@link Coder} for key bytes. + *

Since this changes the type for key, settings that depend on the type + * ({@link #withTimestampFn(SerializableFunction)} and + * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. */ - public Reader withKeyCoder(Coder keyCoder) { + public Read withKeyCoder(Coder keyCoder) { checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); - return new Reader(topics, topicPartitions, keyCoder, valueCoder, null, null, + return new Read(topics, topicPartitions, keyCoder, valueCoder, null, null, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** - * XXX timstampFns after this. + * Set {@link Coder} for value bytes. + *

Since this changes the type for key, settings that depend on the type + * ({@link #withTimestampFn(SerializableFunction)} and + * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. */ - public Reader withValueCoder(Coder valueCoder) { + public Read withValueCoder(Coder valueCoder) { checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); - return new Reader(topics, topicPartitions, keyCoder, valueCoder, null, null, + return new Read(topics, topicPartitions, keyCoder, valueCoder, null, null, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - public Reader withTimestampFn(SerializableFunction, Instant> timestampFn) { + /** + * A function to assign a timestamp to a record. Default is processing timestamp. + */ + public Read withTimestampFn(SerializableFunction, Instant> timestampFn) { checkNotNull(timestampFn); - return new Reader(topics, topicPartitions, keyCoder, valueCoder, + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - public Reader withWatermarkFn(SerializableFunction, Instant> watermarkFn) { + /** + * A function to calculate watermark after a record. Default is last record timestamp + * @see #withTimestampFn(SerializableFunction) + */ + public Read withWatermarkFn(SerializableFunction, Instant> watermarkFn) { checkNotNull(watermarkFn); - return new Reader(topics, topicPartitions, keyCoder, valueCoder, + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - public Reader withConsumerFactoryFn( + /** + * A factory to create Kafka {@link Consumer} from consumer configuration. + * Mainly used for tests. Default factory function creates a {@link KafkaConsumer}. + */ + public Read withConsumerFactoryFn( SerializableFunction, Consumer> consumerFactoryFn) { - return new Reader(topics, topicPartitions, keyCoder, valueCoder, + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - public Reader updateConsumerProperties(Map configUpdates) { + /** + * Update consumer configuration with new properties. + */ + public Read updateConsumerProperties(Map configUpdates) { for (String key : configUpdates.keySet()) { checkArgument(!ignoredConsumerProperties.containsKey(key), "No need to configure '%s'. %s", key, ignoredConsumerProperties.get(key)); @@ -265,16 +293,16 @@ public Reader updateConsumerProperties(Map configUpdates) Map config = new HashMap<>(consumerConfig); config.putAll(configUpdates); - return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, config, maxNumRecords, maxReadTime); } /** - * Similar to {@link Read.Unbounded#withMaxNumRecords(long)}. Mainly used for tests and demo - * applications. + * Similar to {@link Read.Unbounded#withMaxNumRecords(long)}. + * Mainly used for tests and demo applications. */ - public Reader withMaxNumRecords(long maxNumRecords) { - return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + public Read withMaxNumRecords(long maxNumRecords) { + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, null); } @@ -282,17 +310,17 @@ public Reader withMaxNumRecords(long maxNumRecords) { * Similar to {@link Read.Unbounded#withMaxReadTime(Duration)}. Mainly used for tests and demo * applications. */ - public Reader withMaxReadTime(Duration maxReadTime) { - return new Reader(topics, topicPartitions, keyCoder, valueCoder, timestampFn, + public Read withMaxReadTime(Duration maxReadTime) { + return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, Long.MAX_VALUE, maxReadTime); } /** - * XXX NOTE about doing this after setting other properties. - * @return + * A read transform that includes Kafka metadata along with key and value. + * @see {@link KafkaRecord} */ - public ReaderWithMetadata withMetadata() { - return new ReaderWithMetadata(this, + public ReadWithMetadata withMetadata() { + return new ReadWithMetadata(this, timestampFn != null ? unwrapKafkaAndThen(timestampFn) : null, watermarkFn != null ? unwrapKafkaAndThen(watermarkFn) : null); } @@ -331,7 +359,7 @@ public PCollection> apply(PInput input) { * Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. */ PCollection applyHelper(PInput input, UnboundedSource source) { - Read.Unbounded unbounded = Read.from(source); + Unbounded unbounded = com.google.cloud.dataflow.sdk.io.Read.from(source); PTransform> transform = unbounded; if (maxNumRecords < Long.MAX_VALUE) { @@ -344,19 +372,19 @@ PCollection applyHelper(PInput input, UnboundedSource source) { } } - public static class ReaderWithMetadata + public static class ReadWithMetadata extends PTransform>> { - private final Reader kvReader; + private final Read kvRead; @Nullable private final SerializableFunction, Instant> timestampFn; @Nullable private final SerializableFunction, Instant> watermarkFn; - ReaderWithMetadata( - Reader kvReader, + ReadWithMetadata( + Read kvRead, @Nullable SerializableFunction, Instant> timestampFn, @Nullable SerializableFunction, Instant> watermarkFn) { - this.kvReader = kvReader; + this.kvRead = kvRead; this.timestampFn = timestampFn; this.watermarkFn = watermarkFn; } @@ -366,37 +394,37 @@ public static class ReaderWithMetadata // the user to set all those before Reader.withMetaData(). we still need to let users // override timestamp functions (in cases where these functions need metadata). - public ReaderWithMetadata withTimestampFn( + public ReadWithMetadata withTimestampFn( SerializableFunction, Instant> timestampFn) { checkNotNull(timestampFn); - return new ReaderWithMetadata(kvReader, timestampFn, watermarkFn); + return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); } - public ReaderWithMetadata withWatermarkFn( + public ReadWithMetadata withWatermarkFn( SerializableFunction, Instant> watermarkFn) { checkNotNull(watermarkFn); - return new ReaderWithMetadata(kvReader, timestampFn, watermarkFn); + return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); } @VisibleForTesting public UnboundedKafkaSource> makeSource() { return new UnboundedKafkaSource<>( -1, - kvReader.topics, - kvReader.topicPartitions, - kvReader.keyCoder, - kvReader.valueCoder, - KafkaRecordCoder.of(kvReader.keyCoder, kvReader.valueCoder), + kvRead.topics, + kvRead.topicPartitions, + kvRead.keyCoder, + kvRead.valueCoder, + KafkaRecordCoder.of(kvRead.keyCoder, kvRead.valueCoder), new IdentityFn>(), timestampFn, Optional.fromNullable(watermarkFn), - kvReader.consumerFactoryFn, - kvReader.consumerConfig); + kvRead.consumerFactoryFn, + kvRead.consumerConfig); } @Override public PCollection> apply(PInput input) { - return kvReader.applyHelper(input, makeSource()); + return kvRead.applyHelper(input, makeSource()); } } @@ -564,7 +592,7 @@ private static class UnboundedKafkaReader extends UnboundedReader { private static class PartitionState { private final TopicPartition topicPartition; private long consumedOffset; - private Iterator> recordIter = Iterators.emptyIterator(); + private Iterator> recordIter = Collections.emptyIterator(); PartitionState(TopicPartition partition, long offset) { this.topicPartition = partition; @@ -578,7 +606,7 @@ private static class PartitionState { private final List partitionStates; private KafkaRecord curRecord; private Instant curTimestamp; - private Iterator curBatch = Iterators.emptyIterator(); + private Iterator curBatch = Collections.emptyIterator(); /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); @@ -678,7 +706,7 @@ public boolean advance() throws IOException { PartitionState pState = curBatch.next(); if (!pState.recordIter.hasNext()) { // -- (c) - pState.recordIter = Iterators.emptyIterator(); // drop ref + pState.recordIter = Collections.emptyIterator(); // drop ref curBatch.remove(); continue; } diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 02eeaa3436..32b63b0cb3 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -19,7 +19,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Reader; +import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Read; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.io.Read; @@ -163,7 +163,7 @@ private static UnboundedSource, KafkaCheckpointMark> mkKafkaSou List topics = ImmutableList.of("topic_a", "topic_b"); - Reader reader = KafkaIO.reader() + Read reader = KafkaIO.read() .withBootstrapServers("none") .withTopics(topics) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions @@ -236,7 +236,7 @@ public void testUnboundedSourceWithExplicitPartitions() { List topics = ImmutableList.of("test"); - Reader reader = KafkaIO.reader() + Read reader = KafkaIO.read() .withBootstrapServers("none") .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5))) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 10 partitions From 0ea9e7aa028d12f40b2724f6319bdf7950a51152 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 23:00:11 -0800 Subject: [PATCH 47/72] more javadoc --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 62 ++++++++++++++----- .../dataflow/contrib/kafka/KafkaIOTest.java | 8 +-- 2 files changed, 51 insertions(+), 19 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 0084bc63ba..a402cd2075 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -115,11 +115,11 @@ public T apply(T input) { } /** - * Creates and uninitialized {@ Read} transform. Before use, basic Kafka configuration should set - * with {@link Read#withBootstrapServers(String)}, {@link Read#withTopics(List)}. - * Other optional settings include key and value coders, custom timestamp and watermark - * functions. Additionally {@link Read#withMetadata()} provides access to Kafka metadata for - * each record (topic name, partition, and offset). + * Creates and uninitialized {@link Read} {@link PTransform}. Before use, basic Kafka + * configuration should set with {@link Read#withBootstrapServers(String)} and + * {@link Read#withTopics(List)}. Other optional settings include key and value coders, + * custom timestamp and watermark functions. Additionally, {@link Read#withMetadata()} provides + * access to Kafka metadata for each record (topic name, partition, offset). */ public static Read read() { return new Read( @@ -136,8 +136,8 @@ public static Read read() { } /** - * A transform to read from Kafka topics. See {@link KafkaIO#read()} for more information on - * configuration. + * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO#read()} for more + * information on configuration. */ public static class Read extends PTransform>> { @@ -154,7 +154,7 @@ public static class Read extends PTransform>> private final Duration maxReadTime; // bounded read, mainly for testing /** - * set of properties that are not required or don't make sense for our consumer. + * A set of properties that are not required or don't make sense for our consumer. */ private static final Map ignoredConsumerProperties = ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", @@ -206,12 +206,21 @@ public Read( this.maxReadTime = maxReadTime; } + /** + * Returns a new {@link Read} with Kafka consumer pointing to {@code bootstrapServers}. + */ public Read withBootstrapServers(String bootstrapServers) { return updateConsumerProperties( ImmutableMap.of( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)); } + /** + * Returns a new {@link Read} that reads from the topics. All the partitions are from each + * of the topics is read. + * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions) for description + * of how the partitions are distributed among the splits. + */ public Read withTopics(List topics) { checkState(topicPartitions.isEmpty(), "Only topics or topicPartitions can be set, not both"); @@ -219,6 +228,12 @@ public Read withTopics(List topics) { timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } + /** + * Returns a new {@link Read} that reads from the partitions. This allows reading only a subset + * of partitions for one or more topics when (if ever) needed. + * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions) for description + * of how the partitions are distributed among the splits. + */ public Read withTopicPartitions(List topicPartitions) { checkState(topics.isEmpty(), "Only topics or topicPartitions can be set, not both"); @@ -227,7 +242,7 @@ public Read withTopicPartitions(List topicPartitions) { } /** - * Set {@link Coder} for key bytes. + * Returns a new {@link Read} with {@link Coder} for key bytes. *

Since this changes the type for key, settings that depend on the type * ({@link #withTimestampFn(SerializableFunction)} and * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. @@ -240,7 +255,7 @@ public Read withKeyCoder(Coder keyCoder) { } /** - * Set {@link Coder} for value bytes. + * Returns a new {@link Read} with {@link Coder} for value bytes. *

Since this changes the type for key, settings that depend on the type * ({@link #withTimestampFn(SerializableFunction)} and * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. @@ -273,7 +288,7 @@ public Read withWatermarkFn(SerializableFunction, Instant> waterm /** * A factory to create Kafka {@link Consumer} from consumer configuration. - * Mainly used for tests. Default factory function creates a {@link KafkaConsumer}. + * Mainly used for tests. Default is {@link KafkaConsumer}. */ public Read withConsumerFactoryFn( SerializableFunction, Consumer> consumerFactoryFn) { @@ -307,7 +322,8 @@ public Read withMaxNumRecords(long maxNumRecords) { } /** - * Similar to {@link Read.Unbounded#withMaxReadTime(Duration)}. Mainly used for tests and demo + * Similar to {@link Read.Unbounded#withMaxReadTime(Duration)}. + * Mainly used for tests and demo * applications. */ public Read withMaxReadTime(Duration maxReadTime) { @@ -325,6 +341,7 @@ public ReadWithMetadata withMetadata() { watermarkFn != null ? unwrapKafkaAndThen(watermarkFn) : null); } + // utility method to convert KafkRecord to user KV before applying user functions private static SerializableFunction, OutT> unwrapKafkaAndThen(final SerializableFunction, OutT> fn) { return new SerializableFunction, OutT>() { @@ -334,8 +351,13 @@ public OutT apply(KafkaRecord record) { }; } + /** + * Creates an {@link UnboundedSource, ?>} with the configuration in + * {@link ReadWithMetadata}. Primary use case is unit tests, should not be used in an + * application. + */ @VisibleForTesting - public UnboundedKafkaSource> makeSource() { + UnboundedKafkaSource> makeSource() { return new UnboundedKafkaSource>( -1, topics, @@ -394,20 +416,32 @@ public static class ReadWithMetadata // the user to set all those before Reader.withMetaData(). we still need to let users // override timestamp functions (in cases where these functions need metadata). + /** + * A function to assign a timestamp to a record. Default is processing timestamp. + */ public ReadWithMetadata withTimestampFn( SerializableFunction, Instant> timestampFn) { checkNotNull(timestampFn); return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); } + /** + * A function to calculate watermark after a record. Default is last record timestamp + * @see #withTimestampFn(SerializableFunction) + */ public ReadWithMetadata withWatermarkFn( SerializableFunction, Instant> watermarkFn) { checkNotNull(watermarkFn); return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); } + /** + * Creates an {@link UnboundedSource, ?>} with the configuration in + * {@link ReadWithMetadata}. Primary use case is unit tests, should not be used in an + * application. + */ @VisibleForTesting - public UnboundedKafkaSource> makeSource() { + UnboundedKafkaSource> makeSource() { return new UnboundedKafkaSource<>( -1, kvRead.topics, diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 32b63b0cb3..05fa13c552 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Read; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.BigEndianLongCoder; import com.google.cloud.dataflow.sdk.io.Read; @@ -163,7 +162,7 @@ private static UnboundedSource, KafkaCheckpointMark> mkKafkaSou List topics = ImmutableList.of("topic_a", "topic_b"); - Read reader = KafkaIO.read() + KafkaIO.Read reader = KafkaIO.read() .withBootstrapServers("none") .withTopics(topics) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions @@ -219,8 +218,7 @@ public void testUnboundedSource() { int numElements = 1000; PCollection input = p - .apply(Read - .from(mkKafkaSource(numElements, new ValueAsTimestampFn())) + .apply(Read.from(mkKafkaSource(numElements, new ValueAsTimestampFn())) .withMaxNumRecords(numElements)) .apply(Values.create()); @@ -236,7 +234,7 @@ public void testUnboundedSourceWithExplicitPartitions() { List topics = ImmutableList.of("test"); - Read reader = KafkaIO.read() + KafkaIO.Read reader = KafkaIO.read() .withBootstrapServers("none") .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5))) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 10 partitions From b44a201e81a6869e1055ef10c988afb8f62ee075 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 7 Mar 2016 23:39:48 -0800 Subject: [PATCH 48/72] more javadoc --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 48 +++++++++++-------- 1 file changed, 29 insertions(+), 19 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index a402cd2075..07abe940e9 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -73,28 +73,29 @@ import javax.annotation.Nullable; -// {@link Coder} -// generally, feel free to drop @param and @return tags -- relatively unused in google style. -// probably worth javadoccing the splitting behavior extensively here, - /** - * Dataflow Source for consuming Kafka sources. + * Dataflow support for consuming Kafka topics as {@link Unbounded} sources. * - *

- * Usage:
- *        UnboundedSource<String, ?> kafkaSource = KafkaSource
- *            .<String>unboundedValueSourceBuilder()
- *            .withBootstrapServers("broker_1:9092,broker_2:9092)
- *            .withTopics(ImmutableList.of("topic_a", "topic_b")
- *            .withValueCoder(StringUtf8Coder.of())
- *            .withTimestampFn(timestampFn)
- *            .withWatermarkFn(watermarkFn)
- *            .build();
+ * 

+ * The Kafka partitions + * are evenly distributed among splits (workers). Dataflow checkpointing is fully supported and + * each split can resume from previous checkpoint. See + * {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions)} for more details on + * splits and checkpoint support. * - * pipeline - * .apply(Read.from(kafkaSource).named("read_topic_a_and_b")) - * .... - *

+ *
{@code
+ * Usage:
+ *   pipeline
+ *     .apply("read_topic_a_and_b",
+ *        KafkaIO.read()
+ *         .withBootstrapServers("broker_1:9092,broker_2:9092")
+ *         .withTopics(ImmutableList.of("topic_a", "topic_b")) // or withTopicPartitions(List<>)
+ *         .withValueCoder(StringUtf8Coder.of())
+ *         .withTimestampFn(timestampFn) // optional
+ *         .withWatermarkFn(watermarkFn)) // optional
+ *     .apply(Values.create()) // discard keys
+ *     ...
+ *}
*/ public class KafkaIO { @@ -509,6 +510,15 @@ public UnboundedKafkaSource( this.consumerConfig = consumerConfig; } + /** + * The partitions are evenly distributed among the splits. The number of splits returned is + * {@code min(desiredNumSplits, totalNumPartitions)}, though better not to depend on the exact + * count. + * + *

It is important to assign the partitions deterministically so that we can support + * resuming a split from last checkpoint. The Kafka partitions are sorted by + * {@code } and then assigned to splits in round-robin order. + */ @Override public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { From 8db345878d9851a7cfb973ac004686818bb60c4c Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 8 Mar 2016 00:02:57 -0800 Subject: [PATCH 49/72] fix javadoc and checkstyle errors --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 19 ++++++++++++------- .../dataflow/contrib/kafka/KafkaIOTest.java | 2 +- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 07abe940e9..50f6b48b02 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -118,7 +118,7 @@ public T apply(T input) { /** * Creates and uninitialized {@link Read} {@link PTransform}. Before use, basic Kafka * configuration should set with {@link Read#withBootstrapServers(String)} and - * {@link Read#withTopics(List)}. Other optional settings include key and value coders, + * {@link Read#withTopics(List)}. Other optional settings include key and value coders, * custom timestamp and watermark functions. Additionally, {@link Read#withMetadata()} provides * access to Kafka metadata for each record (topic name, partition, offset). */ @@ -183,7 +183,7 @@ public Consumer apply(Map config) { } }; - public Read( + private Read( List topics, List topicPartitions, Coder keyCoder, @@ -219,7 +219,7 @@ public Read withBootstrapServers(String bootstrapServers) { /** * Returns a new {@link Read} that reads from the topics. All the partitions are from each * of the topics is read. - * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions) for description + * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions)} for description * of how the partitions are distributed among the splits. */ public Read withTopics(List topics) { @@ -232,7 +232,7 @@ public Read withTopics(List topics) { /** * Returns a new {@link Read} that reads from the partitions. This allows reading only a subset * of partitions for one or more topics when (if ever) needed. - * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions) for description + * See {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions)} for description * of how the partitions are distributed among the splits. */ public Read withTopicPartitions(List topicPartitions) { @@ -314,7 +314,7 @@ public Read updateConsumerProperties(Map configUpdates) { } /** - * Similar to {@link Read.Unbounded#withMaxNumRecords(long)}. + * Similar to {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded#withMaxNumRecords(long)}. * Mainly used for tests and demo applications. */ public Read withMaxNumRecords(long maxNumRecords) { @@ -323,7 +323,8 @@ public Read withMaxNumRecords(long maxNumRecords) { } /** - * Similar to {@link Read.Unbounded#withMaxReadTime(Duration)}. + * Similar to + * {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded#withMaxReadTime(Duration)}. * Mainly used for tests and demo * applications. */ @@ -334,7 +335,7 @@ public Read withMaxReadTime(Duration maxReadTime) { /** * A read transform that includes Kafka metadata along with key and value. - * @see {@link KafkaRecord} + * @see KafkaRecord */ public ReadWithMetadata withMetadata() { return new ReadWithMetadata(this, @@ -395,6 +396,10 @@ PCollection applyHelper(PInput input, UnboundedSource source) { } } + /** + * Similar to {@link Read}, except that the transform returns a PCollection of + * {@link KafkaRecord} which includes Kafka metadata : topic name, partition, offset. + */ public static class ReadWithMetadata extends PTransform>> { diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 05fa13c552..720e44e78d 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -239,7 +239,7 @@ public void testUnboundedSourceWithExplicitPartitions() { .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5))) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 10 partitions .withValueCoder(BigEndianLongCoder.of()) - .withMaxNumRecords(numElements/10); + .withMaxNumRecords(numElements / 10); PCollection input = p .apply(reader) From 686f608e45586640ab1d356a8468f08df17132c9 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 8 Mar 2016 00:37:00 -0800 Subject: [PATCH 50/72] use KV in KafkaRecord --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- .../dataflow/contrib/kafka/KafkaRecord.java | 30 ++++++++++--------- .../contrib/kafka/KafkaRecordCoder.java | 27 +++++++---------- 3 files changed, 28 insertions(+), 31 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 50f6b48b02..9b5fe49430 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -348,7 +348,7 @@ public ReadWithMetadata withMetadata() { unwrapKafkaAndThen(final SerializableFunction, OutT> fn) { return new SerializableFunction, OutT>() { public OutT apply(KafkaRecord record) { - return fn.apply(KV.of(record.getKey(), record.getValue())); + return fn.apply(record.getKV()); } }; } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index ff40d84b3a..47bd7f21d1 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -18,18 +18,18 @@ import java.io.Serializable; +import com.google.cloud.dataflow.sdk.values.KV; + /** * KafkaRecord contains key and value of the record as well as metadata for the record (topic name, - * partition id, and offset). This is essentially a serializable - * {@link org.apache.kafka.clients.consumer.ConsumerRecord}. + * partition id, and offset). */ public class KafkaRecord implements Serializable { private final String topic; private final int partition; private final long offset; - private final K key; - private final V value; // XXX TODO: use KV instead + private final KV kv; public KafkaRecord( String topic, @@ -37,12 +37,19 @@ public KafkaRecord( long offset, K key, V value) { + this(topic, partition, offset, KV.of(key, value)); + } + + public KafkaRecord( + String topic, + int partition, + long offset, + KV kv) { this.topic = topic; this.partition = partition; this.offset = offset; - this.key = key; - this.value = value; + this.kv = kv; } public String getTopic() { @@ -57,12 +64,8 @@ public long getOffset() { return offset; } - public K getKey() { - return key; - } - - public V getValue() { - return value; + public KV getKV() { + return kv; } @Override @@ -73,8 +76,7 @@ public boolean equals(Object obj) { return topic.equals(other.topic) && partition == other.partition && offset == other.offset - && key.equals(other.key) // XXX KV.equals() - && value.equals(other.value); + && kv.equals(other.kv); } else { return false; } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java index 009916f6fc..d9af1b5659 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecordCoder.java @@ -24,7 +24,7 @@ import com.google.cloud.dataflow.sdk.coders.VarIntCoder; import com.google.cloud.dataflow.sdk.coders.VarLongCoder; import com.google.cloud.dataflow.sdk.util.PropertyNames; -import com.google.common.collect.ImmutableList; +import com.google.cloud.dataflow.sdk.values.KV; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; @@ -43,8 +43,7 @@ public class KafkaRecordCoder extends StandardCoder> { private static final VarLongCoder longCoder = VarLongCoder.of(); private static final VarIntCoder intCoder = VarIntCoder.of(); - private final Coder keyCoder; - private final Coder valueCoder; + private final KvCoder kvCoder; @JsonCreator public static KafkaRecordCoder of(@JsonProperty(PropertyNames.COMPONENT_ENCODINGS) @@ -58,8 +57,7 @@ public static KafkaRecordCoder of(Coder keyCoder, Coder value } public KafkaRecordCoder(Coder keyCoder, Coder valueCoder) { - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; + this.kvCoder = KvCoder.of(keyCoder, valueCoder); } @Override @@ -69,8 +67,7 @@ public void encode(KafkaRecord value, OutputStream outStream, Context cont stringCoder.encode(value.getTopic(), outStream, nested); intCoder.encode(value.getPartition(), outStream, nested); longCoder.encode(value.getOffset(), outStream, nested); - keyCoder.encode(value.getKey(), outStream, nested); - valueCoder.encode(value.getValue(), outStream, nested); + kvCoder.encode(value.getKV(), outStream, nested); } @Override @@ -81,27 +78,26 @@ public KafkaRecord decode(InputStream inStream, Context context) stringCoder.decode(inStream, nested), intCoder.decode(inStream, nested), longCoder.decode(inStream, nested), - keyCoder.decode(inStream, nested), - valueCoder.decode(inStream, nested)); + kvCoder.decode(inStream, nested)); } @Override public List> getCoderArguments() { - return ImmutableList.of(keyCoder, valueCoder); + return kvCoder.getCoderArguments(); } @Override public void verifyDeterministic() throws NonDeterministicException { - verifyDeterministic("Key and Value coder should be deterministic", keyCoder, valueCoder); + kvCoder.verifyDeterministic(); } @Override public boolean isRegisterByteSizeObserverCheap(KafkaRecord value, Context context) { - return keyCoder.isRegisterByteSizeObserverCheap(value.getKey(), context.nested()) - && valueCoder.isRegisterByteSizeObserverCheap(value.getValue(), context.nested()); + return kvCoder.isRegisterByteSizeObserverCheap(value.getKV(), context); //TODO : do we have to implement getEncodedSize()? } + @SuppressWarnings("unchecked") @Override public Object structuralValue(KafkaRecord value) throws Exception { if (consistentWithEquals()) { @@ -111,13 +107,12 @@ public Object structuralValue(KafkaRecord value) throws Exception { value.getTopic(), value.getPartition(), value.getOffset(), - keyCoder.structuralValue(value.getKey()), - valueCoder.structuralValue(value.getValue())); + (KV) kvCoder.structuralValue(value.getKV())); } } @Override public boolean consistentWithEquals() { - return keyCoder.consistentWithEquals() && valueCoder.consistentWithEquals(); + return kvCoder.consistentWithEquals(); } } From dc2ac1abe02790f60f1b4d8568fb137d1be8c071 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 8 Mar 2016 00:40:05 -0800 Subject: [PATCH 51/72] minor --- .../com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java index 47bd7f21d1..584b1b600f 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaRecord.java @@ -16,10 +16,10 @@ package com.google.cloud.dataflow.contrib.kafka; -import java.io.Serializable; - import com.google.cloud.dataflow.sdk.values.KV; +import java.io.Serializable; + /** * KafkaRecord contains key and value of the record as well as metadata for the record (topic name, * partition id, and offset). From b7a9c9104ec8871518d2f8a8ec7b2fce37752aba Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 8 Mar 2016 11:38:25 -0800 Subject: [PATCH 52/72] remove uses of Lists.newArrayList() --- .../google/cloud/dataflow/contrib/kafka/KafkaIO.java | 11 +++++------ .../cloud/dataflow/contrib/kafka/KafkaIOTest.java | 3 +-- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 9b5fe49430..3c44bb8ab4 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -67,6 +67,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; @@ -559,18 +560,16 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { "Could not find any partitions. Please check Kafka configuration and topic names"); int numSplits = Math.min(desiredNumSplits, partitions.size()); - - List> assignments = Lists.newArrayList(); + List> assignments = new ArrayList<>(numSplits); for (int i = 0; i < numSplits; i++) { - assignments.add(Lists.newArrayList()); + assignments.add(new ArrayList()); } - for (int i = 0; i < partitions.size(); i++) { assignments.get(i % numSplits).add(partitions.get(i)); } - List> result = Lists.newArrayList(); + List> result = new ArrayList<>(numSplits); for (int i = 0; i < numSplits; i++) { List assignedToSplit = assignments.get(i); @@ -710,7 +709,7 @@ private void readNextBatch(boolean isFirstFetch) { long timeoutMillis = isFirstFetch ? 4000 : 100; ConsumerRecords records = consumer.poll(timeoutMillis); - List nonEmpty = Lists.newLinkedList(); + List nonEmpty = new LinkedList<>(); for (PartitionState p : partitionStates) { p.recordIter = records.records(p.topicPartition).iterator(); diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 720e44e78d..61dabf5f8b 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -42,7 +42,6 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -87,7 +86,7 @@ private static MockConsumer mkMockConsumer( Map> partitionMap = new HashMap<>(); for (String topic : topics) { - List partIds = Lists.newArrayListWithCapacity(partitionsPerTopic); + List partIds = new ArrayList<>(partitionsPerTopic); for (int i = 0; i < partitionsPerTopic; i++) { partitions.add(new TopicPartition(topic, i)); partIds.add(new PartitionInfo(topic, i, null, null, null)); From fff4a765cb501c45a4a22042255e1e6f7f3b5381 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 10 Mar 2016 15:08:10 -0800 Subject: [PATCH 53/72] minor:formatting fix --- .../kafka/examples/TopHashtagsExample.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 441dc0a49b..d16d42bfc2 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -62,17 +62,17 @@ * containing JSON Tweets, calculates top * hashtags in 10 minute window. The results are written back to a Kafka topic. * - *

+ * 
{@code
  * Usage:
- *   $ java -cp jar_with_dependencies.jar                                   \
- *          com.google.cloud.dataflow.contrib.kafka.TopHashtagsExample      \
- *          --project=GCP_PROJECT                                           \
- *          --stagingLocation=GS_STAGING_DIRECTORY                          \
- *          --runner=BlockingDataflowPipelineRunner                         \
- *          --bootstrapServers="kafka_server_1:9092"                        \
- *          --topics="sample_tweets_json"                                   \
+ *   $ java -cp jar_with_dependencies.jar                                           \
+ *          com.google.cloud.dataflow.contrib.kafka.examples.TopHashtagsExample     \
+ *          --project=GCP_PROJECT                                                   \
+ *          --stagingLocation=GS_STAGING_DIRECTORY                                  \
+ *          --runner=BlockingDataflowPipelineRunner                                 \
+ *          --bootstrapServers="kafka_server_1:9092"                                \
+ *          --topics="sample_tweets_json"                                           \
  *          --outputTopic="top_hashtags"
- * 
+ * }
* */ public class TopHashtagsExample { From 8aea5fac2b9c6a426bcedee4f2cb413cd2a78813 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 15 Mar 2016 17:48:46 -0700 Subject: [PATCH 54/72] update comment for getSplitBacklogBytes --- .../com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 3c44bb8ab4..9ad57a65ce 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -855,9 +855,11 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { @Override public long getSplitBacklogBytes() { - // TODO: fetch latest offsets to estimate backlog. currently looks like we need to pause a - // partition and then seekToEnd() to find the latest offset. - // Hopefully Kafka consumer supports fetching this cleanly. + // TODO: fetch latest offsets to estimate backlog. We could keep an extra consumer open to + // fetch the latest offsets. Maintain an exponential moving average of messages size to + // estimate bytes based on offsets. This is called for every "bundle" and any latency here + // directly adds to processing latency. Better to fetch latest offsets asynchronously. + return super.getSplitBacklogBytes(); } From 4f932705805c03fe0228e16c4435240112484553 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 13:55:14 -0700 Subject: [PATCH 55/72] increase send buffer size for kafka consumer. The default value of 32K severely limits throughput. 100ms polling in KafkaSource and the way Dataflow runner works makes this worse I think. There is about 10x improvement in my tests with fix. --- .../com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 9ad57a65ce..7d6177ca01 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -170,6 +170,15 @@ public static class Read extends PTransform>> ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + + // Use large receive buffer. See KAFKA-3135. may may not be needed in future. + // with default value of of 32K, It takes multiple seconds between successful polls. + // All the consumer work is done inside poll(), with smaller send buffer size, it + // takes many polls before a 1MB chunk from the server is fully read. In my testing + // about half of the time select() inside kafka consumer waits for 20-30ms, though + // the server has lots of data in tcp send buffers on its side. + ConsumerConfig.RECEIVE_BUFFER_CONFIG, 512 * 1024, + // default to latest offset when we are not resuming. ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", // disable auto commit of offsets. we don't require group_id. could be enabled by user. From 432ff26a56e741575235bef65d9f9681fbc47b00 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 16:23:37 -0700 Subject: [PATCH 56/72] poll kafka consumer in a seperate thread to improve throughput. see comment for consumerPollThread --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 70 ++++++++++++++++--- 1 file changed, 59 insertions(+), 11 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 7d6177ca01..5379cb24f8 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -55,6 +55,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.joda.time.Duration; import org.joda.time.Instant; @@ -71,6 +72,10 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; @@ -176,7 +181,8 @@ public static class Read extends PTransform>> // All the consumer work is done inside poll(), with smaller send buffer size, it // takes many polls before a 1MB chunk from the server is fully read. In my testing // about half of the time select() inside kafka consumer waits for 20-30ms, though - // the server has lots of data in tcp send buffers on its side. + // the server has lots of data in tcp send buffers on its side. Also see comment + // about consumerPollThread in UnboundedKafkaReader below. ConsumerConfig.RECEIVE_BUFFER_CONFIG, 512 * 1024, // default to latest offset when we are not resuming. @@ -665,6 +671,15 @@ private static class PartitionState { private Instant curTimestamp; private Iterator curBatch = Collections.emptyIterator(); + // Use a separate thread to read Kafka messages. Kafka Consumer does all its work including + // network I/O inside poll(). Polling only inside #advance(), especially with a small timeout + // like 100 milliseconds does not work well. This along with large receive buffer for + // consumer achieved best throughput in tests (see `defaultConsumerProperties`). + private final ExecutorService consumerPollThread = Executors.newSingleThreadExecutor(); + private final SynchronousQueue> availableRecordsQueue = + new SynchronousQueue<>(); + private volatile boolean closed = false; + /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); @@ -707,16 +722,38 @@ public PartitionState apply(TopicPartition tp) { } } + private void consumerPollLoop() { + // Read in a loop and enqueue the batch of records, if any, to availableRecordsQueue + while (!closed) { + try { + ConsumerRecords records = consumer.poll(1000); + if (!records.isEmpty()) { + availableRecordsQueue.put(records); + } + } catch (InterruptedException e) { + LOG.warn(this + " consumer thread is interrupted", e); // not expected + break; + } catch (WakeupException e) { + break; + } + } + + LOG.info("{} : Returning from consumer pool loop", this); + } + private void readNextBatch(boolean isFirstFetch) { - // read one batch of records with single consumer.poll() (may not return any records) - // Use a longer timeout for first fetch. Kafka consumer seems to do better with poll() with - // longer timeout initially. Looks like it does not handle initial connection setup properly - // with short polls (may also be affected by backoff policy in Dataflow). - // In my tests it took ~5 seconds before first record was read with this - // hack and 20-30 seconds with out. - long timeoutMillis = isFirstFetch ? 4000 : 100; - ConsumerRecords records = consumer.poll(timeoutMillis); + ConsumerRecords records; + try { + records = availableRecordsQueue.poll(10, TimeUnit.MICROSECONDS); + } catch (InterruptedException e) { + LOG.warn("Unexpected", e); + return; + } + + if (records == null) { + return; + } List nonEmpty = new LinkedList<>(); @@ -746,7 +783,14 @@ public boolean start() throws IOException { } } - readNextBatch(true); + // start consumer read loop. + // Note that consumer is not thread safe, should not accessed out side consumerPollLoop() + consumerPollThread.submit(new Runnable() { + public void run() { + consumerPollLoop(); + } + }); + return advance(); } @@ -874,7 +918,11 @@ public long getSplitBacklogBytes() { @Override public void close() throws IOException { - Closeables.close(consumer, true); + closed = true; + availableRecordsQueue.poll(); // drain unread batch, this unblocks consumer thread. + consumer.wakeup(); + consumerPollThread.shutdown(); + Closeables.close(consumer, true); // this should also wake up poll inside consumerPoolLoop. } } } From d408d0b39a0417490cd01d0610ee21f7f3a8189e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 16:31:22 -0700 Subject: [PATCH 57/72] minor --- .../java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 5379cb24f8..73bb3fbee6 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -176,7 +176,7 @@ public static class Read extends PTransform>> ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - // Use large receive buffer. See KAFKA-3135. may may not be needed in future. + // Use large receive buffer. Once KAFKA-3135 is fixed, this not be required. // with default value of of 32K, It takes multiple seconds between successful polls. // All the consumer work is done inside poll(), with smaller send buffer size, it // takes many polls before a 1MB chunk from the server is fully read. In my testing From f0957eb29f890a70976d78a0730e2adf4203feb4 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 16:31:49 -0700 Subject: [PATCH 58/72] minor --- .../java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 73bb3fbee6..e849ddb5e9 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -176,7 +176,7 @@ public static class Read extends PTransform>> ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - // Use large receive buffer. Once KAFKA-3135 is fixed, this not be required. + // Use large receive buffer. Once KAFKA-3135 is fixed, this _may_ not be required. // with default value of of 32K, It takes multiple seconds between successful polls. // All the consumer work is done inside poll(), with smaller send buffer size, it // takes many polls before a 1MB chunk from the server is fully read. In my testing From b531283496c9f6097ea6386dd38eb3d1369124d1 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 17:03:06 -0700 Subject: [PATCH 59/72] micro -> milli --- .../java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index e849ddb5e9..aa5162885d 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -745,7 +745,7 @@ private void readNextBatch(boolean isFirstFetch) { ConsumerRecords records; try { - records = availableRecordsQueue.poll(10, TimeUnit.MICROSECONDS); + records = availableRecordsQueue.poll(10, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { LOG.warn("Unexpected", e); return; From c1f069cafbd4b7b5e27e27114e2a976821adee1e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 17:50:31 -0700 Subject: [PATCH 60/72] minor clean up --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index aa5162885d..936b2c3569 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -741,7 +741,8 @@ private void consumerPollLoop() { LOG.info("{} : Returning from consumer pool loop", this); } - private void readNextBatch(boolean isFirstFetch) { + private void nextBatch() { + curBatch = Collections.emptyIterator(); ConsumerRecords records; try { @@ -797,8 +798,8 @@ public void run() { @Override public boolean advance() throws IOException { /* Read first record (if any). we need to loop here because : - * - (a) some records initially need to be skipped if they are consumedOffset - * - (b) when the current batch empty, we want to readNextBatch() and then advance. + * - (a) some records initially need to be skipped if they are before consumedOffset + * - (b) if curBatch is empty, we want to readNextBatch() and then advance. * - (c) curBatch is an iterator of iterators. we interleave the records from each. * curBatch.next() might return an empty iterator. */ @@ -816,11 +817,10 @@ public boolean advance() throws IOException { long consumed = pState.consumedOffset; long offset = rawRecord.offset(); - // apply user coders if (consumed >= 0 && offset <= consumed) { // -- (a) - // this can happen when compression is enabled in Kafka + // this can happen when compression is enabled in Kafka (seems to be fixed in 0.10) // should we check if the offset is way off from consumedOffset (say > 1M)? - LOG.info("ignoring already consumed offset {} for {}", offset, pState.topicPartition); + LOG.warn("ignoring already consumed offset {} for {}", offset, pState.topicPartition); continue; } @@ -834,7 +834,7 @@ public boolean advance() throws IOException { LOG.info("{} : first record offset {}", name, offset); } - // apply user coders. might want to allow skipping records that fail in coders. + // apply user coders. might want to allow skipping records that fail to decode. curRecord = new KafkaRecord( rawRecord.topic(), rawRecord.partition(), @@ -847,7 +847,7 @@ public boolean advance() throws IOException { return true; } else { // -- (b) - readNextBatch(false); + nextBatch(); if (!curBatch.hasNext()) { return false; @@ -858,9 +858,9 @@ public boolean advance() throws IOException { private static byte[] nullBytes = new byte[0]; private static T decode(byte[] bytes, Coder coder) throws IOException { - // If 'bytes' is null use byte[0]. It is common for key in Kakfa record to be null. + // If 'bytes' is null, use byte[0]. It is common for key in Kakfa record to be null. // This makes it impossible for user to distinguish between zero length byte and null. - // Alternately, we could have a ByteArrayCoder that handles nulls, use that for default + // Alternately, we could have a ByteArrayCoder that handles nulls, and use that for default // coder. byte[] toDecode = bytes == null ? nullBytes : bytes; return coder.decode(new ExposedByteArrayInputStream(toDecode), Coder.Context.OUTER); From 0db0a37226f775d254ff981fe4ac25f1c6e45eff Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 19:47:37 -0700 Subject: [PATCH 61/72] review comments --- .../kafka/examples/TopHashtagsExample.java | 31 ++++++++----------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index d16d42bfc2..8cbbbd9c58 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -17,7 +17,6 @@ package com.google.cloud.dataflow.contrib.kafka.examples; import com.google.cloud.dataflow.contrib.kafka.KafkaIO; -import com.google.cloud.dataflow.contrib.kafka.KafkaIO.Read; import com.google.cloud.dataflow.sdk.Pipeline; import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder; import com.google.cloud.dataflow.sdk.options.Default; @@ -37,7 +36,6 @@ import com.google.cloud.dataflow.sdk.values.KV; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; @@ -54,6 +52,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -73,7 +72,6 @@ * --topics="sample_tweets_json" \ * --outputTopic="top_hashtags" * } - * */ public class TopHashtagsExample { @@ -83,22 +81,22 @@ public class TopHashtagsExample { * Options for the app. */ public static interface Options extends PipelineOptions { - @Description("Sliding window size, in minutes") + @Description("Sliding window length in minutes") @Default.Integer(10) - Integer getSlidingWindowSize(); - void setSlidingWindowSize(Integer value); + Integer getSlidingWindowLengthMinutes(); + void setSlidingWindowLengthMinutes(Integer value); - @Description("Trigger window period, in minutes") + @Description("Trigger window interval in minutes") @Default.Integer(1) - Integer getSlidingWindowPeriod(); - void setSlidingWindowPeriod(Integer value); + Integer getSlidingWindowIntervalMinutes(); + void setSlidingWindowIntervalMinutes(Integer value); - @Description("Bootstarp Server(s) for Kafka") + @Description("Bootstrap Server(s) for Kafka") @Required String getBootstrapServers(); void setBootstrapServers(String servers); - @Description("One or more topics to read from") + @Description("One or more comma seperated topics to read from") @Required List getTopics(); void setTopics(List topics); @@ -120,10 +118,7 @@ public static void main(String args[]) { Pipeline pipeline = Pipeline.create(options); - final int windowSize = options.getSlidingWindowSize(); - final int windowPeriod = options.getSlidingWindowPeriod(); - - Read reader = KafkaIO.read() + KafkaIO.Read reader = KafkaIO.read() .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) .withValueCoder(StringUtf8Coder.of()) @@ -134,8 +129,8 @@ public static void main(String args[]) { .apply(Values.create()) .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows - .of(Duration.standardMinutes(windowSize)) - .every(Duration.standardMinutes(windowPeriod)))) + .of(Duration.standardMinutes(options.getSlidingWindowLengthMinutes())) + .every(Duration.standardMinutes(options.getSlidingWindowIntervalMinutes())))) .apply(Count.perElement()) .apply(Top.of(options.getNumTopHashtags(), new KV.OrderByValue()) .withoutDefaults()) @@ -218,7 +213,7 @@ public void processElement(ProcessContext ctx) throws Exception { jsonWriter = new ObjectMapper().writerWithType(OutputJson.class); } - List topHashtags = Lists.newArrayListWithCapacity(ctx.element().size()); + List topHashtags = new ArrayList<>(ctx.element().size()); for (KV tag : ctx.element()) { topHashtags.add(new HashtagInfo(tag.getKey(), tag.getValue())); From c2b4c3e48b4afb56460dcf6ac1555a65f8ab349e Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 17 Mar 2016 23:55:02 -0700 Subject: [PATCH 62/72] fetch latest offsets in a thread. some review somments --- .../kafka/examples/TopHashtagsExample.java | 9 +- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 137 ++++++++++++++---- .../dataflow/contrib/kafka/KafkaIOTest.java | 3 + 3 files changed, 117 insertions(+), 32 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 8cbbbd9c58..61dd0fe425 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -140,9 +140,10 @@ public static void main(String args[]) { pipeline.run(); } - // The rest of the file implements DoFns to extract hashtags, formatting output, writing output - // back to Kafka. Note that writing to Kafka is not a complete Dataflow Sink. It is a best-effort - // logging of the results. + // The rest of the file implements DoFns to do the following: + // - extract hashtags + // - format results in json + // - write the results back to Kafka (useful for fetching monitoring the end result). private static final ObjectMapper jsonMapper = new ObjectMapper(); @@ -250,7 +251,7 @@ public void processElement(ProcessContext ctx) throws Exception { if (producer == null) { producer = new KafkaProducer(config); } - LOG.info("Top Hashtags : " + ctx.element()); + LOG.info("Top Hashtags : {}", ctx.element()); producer.send(new ProducerRecord(topic, ctx.element())); } } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 936b2c3569..1b2728a785 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -74,6 +74,7 @@ import java.util.NoSuchElementException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.TimeUnit; @@ -180,9 +181,9 @@ public static class Read extends PTransform>> // with default value of of 32K, It takes multiple seconds between successful polls. // All the consumer work is done inside poll(), with smaller send buffer size, it // takes many polls before a 1MB chunk from the server is fully read. In my testing - // about half of the time select() inside kafka consumer waits for 20-30ms, though - // the server has lots of data in tcp send buffers on its side. Also see comment - // about consumerPollThread in UnboundedKafkaReader below. + // about half of the time select() inside kafka consumer waited for 20-30ms, though + // the server had lots of data in tcp send buffers on its side. Compared to default, + // this setting increased throughput increased by many fold (3-4x). ConsumerConfig.RECEIVE_BUFFER_CONFIG, 512 * 1024, // default to latest offset when we are not resuming. @@ -651,21 +652,10 @@ public Coder getDefaultOutputCoder() { private static class UnboundedKafkaReader extends UnboundedReader { - // maintains state of each assigned partition (buffered records and consumed offset) - private static class PartitionState { - private final TopicPartition topicPartition; - private long consumedOffset; - private Iterator> recordIter = Collections.emptyIterator(); - - PartitionState(TopicPartition partition, long offset) { - this.topicPartition = partition; - this.consumedOffset = offset; - } - } - private final UnboundedKafkaSource source; private final String name; private Consumer consumer; + private Consumer secondaryConsumer; // for fetching latest offsets private final List partitionStates; private KafkaRecord curRecord; private Instant curTimestamp; @@ -680,6 +670,10 @@ private static class PartitionState { new SynchronousQueue<>(); private volatile boolean closed = false; + // thread used to run a periodic task to fetch latest offsets. + private final ScheduledExecutorService offsetFetcherThread = + Executors.newSingleThreadScheduledExecutor(); + /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); @@ -687,6 +681,54 @@ public String toString() { return name; } + // maintains state of each assigned partition (buffered records, consumed offset, etc) + private static class PartitionState { + private final TopicPartition topicPartition; + private long consumedOffset; + private long latestOffset; + private Iterator> recordIter = Collections.emptyIterator(); + + // simple moving average for size of each record in bytes + private double avgRecordSize = 0; + private static final int movingAvgWindow = 1000; // very roughly avg of last 1000 elements + + + PartitionState(TopicPartition partition, long offset) { + this.topicPartition = partition; + this.consumedOffset = offset; + this.latestOffset = -1; + } + + // update consumedOffset and avgRecordSize + void recordConsumed(long offset, int size) { + consumedOffset = offset; + + // this is always updated from single thread. probably not worth making it an AtomicDouble + if (avgRecordSize <= 0) { + avgRecordSize = size; + } else { + // initially, first record heavily contributes to average. + avgRecordSize += ((size - avgRecordSize) / movingAvgWindow); + } + } + + synchronized void setLatestOffset(long latestOffset) { + this.latestOffset = latestOffset; + } + + synchronized long approxBacklogInBytes() { + // Note that is an an estimate of uncompressed backlog. + // Messages on Kafka might be comressed. + if (latestOffset < 0 || consumedOffset < 0) { + return UnboundedReader.BACKLOG_UNKNOWN; + } + if (latestOffset <= consumedOffset || consumedOffset < 0) { + return 0; + } + return (long) ((latestOffset - consumedOffset - 1) * avgRecordSize); + } + } + public UnboundedKafkaReader( UnboundedKafkaSource source, @Nullable KafkaCheckpointMark checkpointMark) { @@ -728,7 +770,7 @@ private void consumerPollLoop() { try { ConsumerRecords records = consumer.poll(1000); if (!records.isEmpty()) { - availableRecordsQueue.put(records); + availableRecordsQueue.put(records); // blocks until dequeued. } } catch (InterruptedException e) { LOG.warn(this + " consumer thread is interrupted", e); // not expected @@ -772,7 +814,10 @@ private void nextBatch() { @Override public boolean start() throws IOException { consumer = source.consumerFactoryFn.apply(source.consumerConfig); + secondaryConsumer = source.consumerFactoryFn.apply(source.consumerConfig); + consumer.assign(source.assignedPartitions); + secondaryConsumer.assign(source.assignedPartitions); // seek to consumedOffset + 1 if it is set for (PartitionState p : partitionStates) { @@ -786,11 +831,20 @@ public boolean start() throws IOException { // start consumer read loop. // Note that consumer is not thread safe, should not accessed out side consumerPollLoop() - consumerPollThread.submit(new Runnable() { - public void run() { - consumerPollLoop(); - } - }); + consumerPollThread.submit( + new Runnable() { + public void run() { + consumerPollLoop(); + } + }); + + // update latest offsets every 5 seconds + offsetFetcherThread.scheduleAtFixedRate( + new Runnable() { + public void run() { + updateLatestOffsets(); + } + }, 0, 5, TimeUnit.SECONDS); return advance(); } @@ -843,7 +897,10 @@ public boolean advance() throws IOException { decode(rawRecord.value(), source.valueCoder)); curTimestamp = source.timestampFn.apply(curRecord); - pState.consumedOffset = offset; + + int recordSize = (rawRecord.key() == null ? 0 : rawRecord.key().length) + + (rawRecord.value() == null ? 0 : rawRecord.value().length); + pState.recordConsumed(offset, recordSize); return true; } else { // -- (b) @@ -866,6 +923,23 @@ private static T decode(byte[] bytes, Coder coder) throws IOException { return coder.decode(new ExposedByteArrayInputStream(toDecode), Coder.Context.OUTER); } + // called from offsetFetcher thread + private void updateLatestOffsets() { + for (PartitionState p : partitionStates) { + try { + secondaryConsumer.seekToEnd(p.topicPartition); + long offset = secondaryConsumer.position(p.topicPartition); + p.setLatestOffset(offset);; + } catch (Exception e) { + LOG.warn(this + " : exception while fetching latest offsets. ignored.", e); + p.setLatestOffset(-1L); // reset + } + + LOG.debug("{} : latest offset update for {} : {} (consumed offset {}, avg record size {})", + this, p.topicPartition, p.latestOffset, p.consumedOffset, p.avgRecordSize); + } + } + @Override public Instant getWatermark() { if (curRecord == null) { @@ -908,12 +982,17 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { @Override public long getSplitBacklogBytes() { - // TODO: fetch latest offsets to estimate backlog. We could keep an extra consumer open to - // fetch the latest offsets. Maintain an exponential moving average of messages size to - // estimate bytes based on offsets. This is called for every "bundle" and any latency here - // directly adds to processing latency. Better to fetch latest offsets asynchronously. + long backlogBytes = 0; + for (PartitionState p : partitionStates) { + long pBacklog = p.approxBacklogInBytes(); + if (pBacklog == UnboundedReader.BACKLOG_UNKNOWN) { + return UnboundedReader.BACKLOG_UNKNOWN; + } + backlogBytes += pBacklog; + } - return super.getSplitBacklogBytes(); + LOG.info("{} backlog reported : {}", this, backlogBytes); + return backlogBytes; } @Override @@ -922,7 +1001,9 @@ public void close() throws IOException { availableRecordsQueue.poll(); // drain unread batch, this unblocks consumer thread. consumer.wakeup(); consumerPollThread.shutdown(); - Closeables.close(consumer, true); // this should also wake up poll inside consumerPoolLoop. + offsetFetcherThread.shutdown(); + Closeables.close(secondaryConsumer, true); + Closeables.close(consumer, true); } } } diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index 61dabf5f8b..c5333adf27 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -42,6 +42,7 @@ import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PCollectionList; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -105,6 +106,7 @@ private static MockConsumer mkMockConsumer( records.put(tp, new ArrayList>()); } records.get(tp).add( + // Note: this interface has changed in 0.10. may get fixed before the release. new ConsumerRecord( tp.topic(), tp.partition(), @@ -122,6 +124,7 @@ public void assign(List assigned) { for (ConsumerRecord r : records.get(tp)) { addRecord(r); } + updateBeginningOffsets(ImmutableMap.of(tp, 0L)); seek(tp, 0); } } From e1b0d0d5e598c51c8a5fe91b6851ed3f5b50511a Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 18 Mar 2016 07:27:28 -0700 Subject: [PATCH 63/72] offset consumer : comment on hack, disable auto_offset_commit --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 41 +++++++++++++------ 1 file changed, 29 insertions(+), 12 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 1b2728a785..c73ebb632e 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -72,6 +72,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -655,7 +656,6 @@ private static class UnboundedKafkaReader extends UnboundedReader { private final UnboundedKafkaSource source; private final String name; private Consumer consumer; - private Consumer secondaryConsumer; // for fetching latest offsets private final List partitionStates; private KafkaRecord curRecord; private Instant curTimestamp; @@ -670,9 +670,17 @@ private static class UnboundedKafkaReader extends UnboundedReader { new SynchronousQueue<>(); private volatile boolean closed = false; - // thread used to run a periodic task to fetch latest offsets. + // Backlog support : + // Kafka consumer does not have an API to fetch latest offset for topic. We need to seekToEnd() + // then look at position(). Use another consumer to do this so that the primary consumer does + // not need to be interrupted. The latest offsets are fetched periodically on another thread. + // This is still a hack. There could be unintended side effects, e.g. if user enabled offset + // auto commit in consumer config, this could interfere with the primary consumer (we will + // handle this particular problem). We might have to make this optional. + private Consumer offsetConsumer; private final ScheduledExecutorService offsetFetcherThread = Executors.newSingleThreadScheduledExecutor(); + static private final int OFFSET_UPDATE_INTERVAL_MINUTES = 5; /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); @@ -814,10 +822,7 @@ private void nextBatch() { @Override public boolean start() throws IOException { consumer = source.consumerFactoryFn.apply(source.consumerConfig); - secondaryConsumer = source.consumerFactoryFn.apply(source.consumerConfig); - consumer.assign(source.assignedPartitions); - secondaryConsumer.assign(source.assignedPartitions); // seek to consumedOffset + 1 if it is set for (PartitionState p : partitionStates) { @@ -838,13 +843,25 @@ public void run() { } }); - // update latest offsets every 5 seconds + // offsetConsumer setup : + + // override client_id and auto_commit so that it does not interfere with main consumer. + String offsetConsumerId = String.format("%s_offset_consumer_%d_%s", name, + (new Random()).nextInt(Integer.MAX_VALUE), + source.consumerConfig.getOrDefault(ConsumerConfig.CLIENT_ID_CONFIG, "none")); + Map offsetConsumerConfig = new HashMap<>(source.consumerConfig); + offsetConsumerConfig.put(ConsumerConfig.CLIENT_ID_CONFIG, offsetConsumerId); + offsetConsumerConfig.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + offsetConsumer = source.consumerFactoryFn.apply(offsetConsumerConfig); + offsetConsumer.assign(source.assignedPartitions); + offsetFetcherThread.scheduleAtFixedRate( new Runnable() { public void run() { updateLatestOffsets(); } - }, 0, 5, TimeUnit.SECONDS); + }, 0, OFFSET_UPDATE_INTERVAL_MINUTES, TimeUnit.SECONDS); return advance(); } @@ -853,9 +870,9 @@ public void run() { public boolean advance() throws IOException { /* Read first record (if any). we need to loop here because : * - (a) some records initially need to be skipped if they are before consumedOffset - * - (b) if curBatch is empty, we want to readNextBatch() and then advance. + * - (b) if curBatch is empty, we want to fetch next batch and then advance. * - (c) curBatch is an iterator of iterators. we interleave the records from each. - * curBatch.next() might return an empty iterator. + * curBatch.next() might return an empty iterator. */ while (true) { if (curBatch.hasNext()) { @@ -927,8 +944,8 @@ private static T decode(byte[] bytes, Coder coder) throws IOException { private void updateLatestOffsets() { for (PartitionState p : partitionStates) { try { - secondaryConsumer.seekToEnd(p.topicPartition); - long offset = secondaryConsumer.position(p.topicPartition); + offsetConsumer.seekToEnd(p.topicPartition); + long offset = offsetConsumer.position(p.topicPartition); p.setLatestOffset(offset);; } catch (Exception e) { LOG.warn(this + " : exception while fetching latest offsets. ignored.", e); @@ -1002,7 +1019,7 @@ public void close() throws IOException { consumer.wakeup(); consumerPollThread.shutdown(); offsetFetcherThread.shutdown(); - Closeables.close(secondaryConsumer, true); + Closeables.close(offsetConsumer, true); Closeables.close(consumer, true); } } From 25609caf4f6b3212efa95749f4cb6ad26ef20bef Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 18 Mar 2016 07:34:24 -0700 Subject: [PATCH 64/72] minor --- .../java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index c73ebb632e..6f2f6c8046 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -955,6 +955,8 @@ private void updateLatestOffsets() { LOG.debug("{} : latest offset update for {} : {} (consumed offset {}, avg record size {})", this, p.topicPartition, p.latestOffset, p.consumedOffset, p.avgRecordSize); } + + LOG.debug("{} : backlog {}", getSplitBacklogBytes()); } @Override @@ -1008,7 +1010,6 @@ public long getSplitBacklogBytes() { backlogBytes += pBacklog; } - LOG.info("{} backlog reported : {}", this, backlogBytes); return backlogBytes; } From ecc6376d9d5c3fd1e17032a7fa0fc540b65714da Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 18 Mar 2016 08:12:49 -0700 Subject: [PATCH 65/72] minutes -> seconds --- .../com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 6f2f6c8046..57e5435142 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -680,7 +680,7 @@ private static class UnboundedKafkaReader extends UnboundedReader { private Consumer offsetConsumer; private final ScheduledExecutorService offsetFetcherThread = Executors.newSingleThreadScheduledExecutor(); - static private final int OFFSET_UPDATE_INTERVAL_MINUTES = 5; + static private final int OFFSET_UPDATE_INTERVAL_SECONDS = 5; /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); @@ -861,7 +861,7 @@ public void run() { public void run() { updateLatestOffsets(); } - }, 0, OFFSET_UPDATE_INTERVAL_MINUTES, TimeUnit.SECONDS); + }, 0, OFFSET_UPDATE_INTERVAL_SECONDS, TimeUnit.SECONDS); return advance(); } @@ -956,7 +956,7 @@ private void updateLatestOffsets() { this, p.topicPartition, p.latestOffset, p.consumedOffset, p.avgRecordSize); } - LOG.debug("{} : backlog {}", getSplitBacklogBytes()); + LOG.debug("{} : backlog {}", this, getSplitBacklogBytes()); } @Override From 43973e72d38d4218d4560517a2e5e1582c7b8798 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Fri, 18 Mar 2016 11:44:37 -0700 Subject: [PATCH 66/72] fix warning in tests. a few more minor diffs --- .../contrib/kafka/examples/TopHashtagsExample.java | 12 ++++++++---- .../google/cloud/dataflow/contrib/kafka/KafkaIO.java | 2 +- .../cloud/dataflow/contrib/kafka/KafkaIOTest.java | 1 + 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 61dd0fe425..6632685e21 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -155,8 +155,8 @@ private static class ExtractHashtagsFn extends DoFn { @Override public void processElement(ProcessContext ctx) throws Exception { for (JsonNode hashtag : jsonMapper.readTree(ctx.element()) - .with("entities") - .withArray("hashtags")) { + .with("entities") + .withArray("hashtags")) { ctx.output(hashtag.get("text").asText()); } } @@ -247,10 +247,14 @@ public KafkaWriter(Options options) { } @Override - public void processElement(ProcessContext ctx) throws Exception { - if (producer == null) { + public void startBundle(Context c) throws Exception { + if (producer == null) { // in Beam, startBundle might be called multiple times. producer = new KafkaProducer(config); } + } + + @Override + public void processElement(ProcessContext ctx) throws Exception { LOG.info("Top Hashtags : {}", ctx.element()); producer.send(new ProducerRecord(topic, ctx.element())); } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 57e5435142..a075ec2a06 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -680,7 +680,7 @@ private static class UnboundedKafkaReader extends UnboundedReader { private Consumer offsetConsumer; private final ScheduledExecutorService offsetFetcherThread = Executors.newSingleThreadScheduledExecutor(); - static private final int OFFSET_UPDATE_INTERVAL_SECONDS = 5; + private static final int OFFSET_UPDATE_INTERVAL_SECONDS = 5; /** watermark before any records have been read. */ private static Instant initialWatermark = new Instant(Long.MIN_VALUE); diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index c5333adf27..bb80f61eac 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -125,6 +125,7 @@ public void assign(List assigned) { addRecord(r); } updateBeginningOffsets(ImmutableMap.of(tp, 0L)); + updateEndOffsets(ImmutableMap.of(tp, (long)records.get(tp).size())); seek(tp, 0); } } From 32e7362c54f503d7e18f66fee78e5e8abef3a4ae Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 21 Mar 2016 22:05:26 -0700 Subject: [PATCH 67/72] review comments --- .../kafka/examples/TopHashtagsExample.java | 42 ++++++++++--------- .../contrib/kafka/KafkaCheckpointMark.java | 18 ++++---- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 16 +++---- 3 files changed, 39 insertions(+), 37 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 6632685e21..219db5deb5 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -96,7 +96,7 @@ public static interface Options extends PipelineOptions { String getBootstrapServers(); void setBootstrapServers(String servers); - @Description("One or more comma seperated topics to read from") + @Description("One or more comma separated topics to read from") @Required List getTopics(); void setTopics(List topics); @@ -122,10 +122,10 @@ public static void main(String args[]) { .withBootstrapServers(options.getBootstrapServers()) .withTopics(options.getTopics()) .withValueCoder(StringUtf8Coder.of()) - .withTimestampFn(timestampFn); + .withTimestampFn(TWEET_TIMESTAMP_OR_NOW); pipeline - .apply("sample_tweets", reader) + .apply(reader) .apply(Values.create()) .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows @@ -145,7 +145,7 @@ public static void main(String args[]) { // - format results in json // - write the results back to Kafka (useful for fetching monitoring the end result). - private static final ObjectMapper jsonMapper = new ObjectMapper(); + private static final ObjectMapper JSON_MAPPER = new ObjectMapper(); /** * Emit hashtags in the tweet (if any). @@ -154,21 +154,21 @@ private static class ExtractHashtagsFn extends DoFn { @Override public void processElement(ProcessContext ctx) throws Exception { - for (JsonNode hashtag : jsonMapper.readTree(ctx.element()) - .with("entities") - .withArray("hashtags")) { + for (JsonNode hashtag : JSON_MAPPER.readTree(ctx.element()) + .with("entities") + .withArray("hashtags")) { ctx.output(hashtag.get("text").asText()); } } } // extract timestamp from "timestamp_ms" field. - private static SerializableFunction, Instant> timestampFn = + private static final SerializableFunction, Instant> TWEET_TIMESTAMP_OR_NOW = new SerializableFunction, Instant>() { @Override public Instant apply(KV kv) { try { - long tsMillis = jsonMapper.readTree(kv.getValue()).path("timestamp_ms").asLong(); + long tsMillis = JSON_MAPPER.readTree(kv.getValue()).path("timestamp_ms").asLong(); return tsMillis == 0 ? Instant.now() : new Instant(tsMillis); } catch (Exception e) { throw Throwables.propagate(e); @@ -180,8 +180,11 @@ public Instant apply(KV kv) { private static class OutputFormatter extends DoFn>, String> implements DoFn.RequiresWindowAccess { - private transient DateTimeFormatter formatter; - private transient ObjectWriter jsonWriter; + private static final DateTimeFormatter DATE_FORMATTER = DateTimeFormat + .forPattern("yyyy-MM-dd HH:mm:ss") + .withZoneUTC(); + private static final ObjectWriter JSON_WRITER = new ObjectMapper() + .writerWithType(OutputJson.class); static class OutputJson { @JsonProperty String windowStart; @@ -209,10 +212,6 @@ static class HashtagInfo { @Override public void processElement(ProcessContext ctx) throws Exception { - if (formatter == null) { - formatter = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss").withZoneUTC(); - jsonWriter = new ObjectMapper().writerWithType(OutputJson.class); - } List topHashtags = new ArrayList<>(ctx.element().size()); @@ -222,10 +221,10 @@ public void processElement(ProcessContext ctx) throws Exception { IntervalWindow window = (IntervalWindow) ctx.window(); - String json = jsonWriter.writeValueAsString(new OutputJson( - formatter.print(window.start()), - formatter.print(window.end()), - formatter.print(Instant.now()), + String json = JSON_WRITER.writeValueAsString(new OutputJson( + DATE_FORMATTER.print(window.start()), + DATE_FORMATTER.print(window.end()), + DATE_FORMATTER.print(Instant.now()), topHashtags)); ctx.output(json); @@ -253,6 +252,11 @@ public void startBundle(Context c) throws Exception { } } + @Override + public void finishBundle(Context c) throws Exception { + producer.close(); + } + @Override public void processElement(ProcessContext ctx) throws Exception { LOG.info("Top Hashtags : {}", ctx.element()); diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java index 804c2a6910..9b33ee809c 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaCheckpointMark.java @@ -27,7 +27,7 @@ import java.util.List; /** - * Checkpoint for an unbounded KafkaSource reader. Consists of Kafka topic name, partition id, + * Checkpoint for an unbounded KafkaIO.Read. Consists of Kafka topic name, partition id, * and the latest offset consumed so far. */ @DefaultCoder(SerializableCoder.class) @@ -45,19 +45,15 @@ public List getPartitions() { @Override public void finalizeCheckpoint() throws IOException { - /* - * nothing to do. - * - * we might want to support committing offset in Kafka, though it does not guarantee - * no-duplicates, it could support Dataflow restart better. Unlike an update of a dataflow job, - * a restart does not have checkpoint state. This secondary checkpoint might be a good start - * for readers. Another similar benefit is when the number of workers or number of - * Kafka partitions changes. - */ + /* nothing to do */ + + // We might want to support committing offset in Kafka for better resume point when the job + // is restarted (checkpoint is not available for job restarts). } /** - * TopicPartition, offset tuple. Defines specific location in the partitions. + * A tuple to hold topic, partition, and offset that comprise the checkpoint + * for a single partition. */ public static class PartitionMark implements Serializable { private final TopicPartition topicPartition; diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index a075ec2a06..ea1de70229 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -138,8 +138,8 @@ public static Read read() { ByteArrayCoder.of(), null, null, - Read.kafka9ConsumerFactory, - Read.defaultConsumerProperties, + Read.KAFKA_9_CONSUMER_FACTORY_FN, + Read.DEFAULT_CONSUMER_PROPERTIES, Long.MAX_VALUE, null); } @@ -173,7 +173,7 @@ public static class Read extends PTransform>> ); // set config defaults - private static final Map defaultConsumerProperties = + private static final Map DEFAULT_CONSUMER_PROPERTIES = ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), @@ -192,9 +192,9 @@ public static class Read extends PTransform>> // disable auto commit of offsets. we don't require group_id. could be enabled by user. ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); - // default Kafka 0.9 Consumer supplier. static variable to avoid capturing 'this' - private static SerializableFunction, Consumer> - kafka9ConsumerFactory = + // default Kafka 0.9 Consumer supplier. + private static final SerializableFunction, Consumer> + KAFKA_9_CONSUMER_FACTORY_FN = new SerializableFunction, Consumer>() { public Consumer apply(Map config) { return new KafkaConsumer<>(config); @@ -307,7 +307,8 @@ public Read withWatermarkFn(SerializableFunction, Instant> waterm /** * A factory to create Kafka {@link Consumer} from consumer configuration. - * Mainly used for tests. Default is {@link KafkaConsumer}. + * This is useful for supporting another version of Kafka consumer. + * Default is {@link KafkaConsumer}. */ public Read withConsumerFactoryFn( SerializableFunction, Consumer> consumerFactoryFn) { @@ -1002,6 +1003,7 @@ public Instant getCurrentTimestamp() throws NoSuchElementException { @Override public long getSplitBacklogBytes() { long backlogBytes = 0; + for (PartitionState p : partitionStates) { long pBacklog = p.approxBacklogInBytes(); if (pBacklog == UnboundedReader.BACKLOG_UNKNOWN) { From 70a0093fdcde621ad4c35c3d9a3080bfe15892f6 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 31 Mar 2016 11:04:26 -0700 Subject: [PATCH 68/72] Update KafkaIO interface based on sugestions from Dan. This certainly looks much better. --- .../kafka/examples/TopHashtagsExample.java | 13 +- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 459 +++++++++--------- .../dataflow/contrib/kafka/KafkaIOTest.java | 49 +- 3 files changed, 253 insertions(+), 268 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index 219db5deb5..b6b49c9d15 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -118,14 +118,13 @@ public static void main(String args[]) { Pipeline pipeline = Pipeline.create(options); - KafkaIO.Read reader = KafkaIO.read() - .withBootstrapServers(options.getBootstrapServers()) - .withTopics(options.getTopics()) - .withValueCoder(StringUtf8Coder.of()) - .withTimestampFn(TWEET_TIMESTAMP_OR_NOW); - pipeline - .apply(reader) + .apply(KafkaIO.read() + .withBootstrapServers(options.getBootstrapServers()) + .withTopics(options.getTopics()) + .withValueCoder(StringUtf8Coder.of()) + .withTimestampFn(TWEET_TIMESTAMP_OR_NOW) + .withoutMetadata()) .apply(Values.create()) .apply(ParDo.of(new ExtractHashtagsFn())) .apply(Window.into(SlidingWindows diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index ea1de70229..5d3c503ad0 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -24,17 +24,19 @@ import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; -import com.google.cloud.dataflow.sdk.coders.KvCoder; import com.google.cloud.dataflow.sdk.coders.SerializableCoder; import com.google.cloud.dataflow.sdk.io.Read.Unbounded; import com.google.cloud.dataflow.sdk.io.UnboundedSource; import com.google.cloud.dataflow.sdk.io.UnboundedSource.CheckpointMark; import com.google.cloud.dataflow.sdk.io.UnboundedSource.UnboundedReader; import com.google.cloud.dataflow.sdk.options.PipelineOptions; +import com.google.cloud.dataflow.sdk.transforms.DoFn; import com.google.cloud.dataflow.sdk.transforms.PTransform; +import com.google.cloud.dataflow.sdk.transforms.ParDo; import com.google.cloud.dataflow.sdk.transforms.SerializableFunction; import com.google.cloud.dataflow.sdk.util.ExposedByteArrayInputStream; import com.google.cloud.dataflow.sdk.values.KV; +import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; import com.google.common.base.Function; @@ -116,12 +118,6 @@ public Instant apply(T input) { } } - private static class IdentityFn implements SerializableFunction { - @Override - public T apply(T input) { - return input; - } - } /** * Creates and uninitialized {@link Read} {@link PTransform}. Before use, basic Kafka @@ -136,8 +132,6 @@ public static Read read() { new ArrayList(), ByteArrayCoder.of(), ByteArrayCoder.of(), - null, - null, Read.KAFKA_9_CONSUMER_FACTORY_FN, Read.DEFAULT_CONSUMER_PROPERTIES, Long.MAX_VALUE, @@ -145,85 +139,10 @@ public static Read read() { } /** - * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO#read()} for more - * information on configuration. + * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more + * information on usage and configuration. */ - public static class Read extends PTransform>> { - - private final List topics; - private final List topicPartitions; // mutually exclusive with topics - private final Coder keyCoder; - private final Coder valueCoder; - @Nullable private final SerializableFunction, Instant> timestampFn; - @Nullable private final SerializableFunction, Instant> watermarkFn; - private final - SerializableFunction, Consumer> consumerFactoryFn; - private final Map consumerConfig; - private final long maxNumRecords; // bounded read, mainly for testing - private final Duration maxReadTime; // bounded read, mainly for testing - - /** - * A set of properties that are not required or don't make sense for our consumer. - */ - private static final Map ignoredConsumerProperties = ImmutableMap.of( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDecoderFn instead" - // "group.id", "enable.auto.commit", "auto.commit.interval.ms" : - // lets allow these, applications can have better resume point for restarts. - ); - - // set config defaults - private static final Map DEFAULT_CONSUMER_PROPERTIES = - ImmutableMap.of( - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), - - // Use large receive buffer. Once KAFKA-3135 is fixed, this _may_ not be required. - // with default value of of 32K, It takes multiple seconds between successful polls. - // All the consumer work is done inside poll(), with smaller send buffer size, it - // takes many polls before a 1MB chunk from the server is fully read. In my testing - // about half of the time select() inside kafka consumer waited for 20-30ms, though - // the server had lots of data in tcp send buffers on its side. Compared to default, - // this setting increased throughput increased by many fold (3-4x). - ConsumerConfig.RECEIVE_BUFFER_CONFIG, 512 * 1024, - - // default to latest offset when we are not resuming. - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", - // disable auto commit of offsets. we don't require group_id. could be enabled by user. - ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); - - // default Kafka 0.9 Consumer supplier. - private static final SerializableFunction, Consumer> - KAFKA_9_CONSUMER_FACTORY_FN = - new SerializableFunction, Consumer>() { - public Consumer apply(Map config) { - return new KafkaConsumer<>(config); - } - }; - - private Read( - List topics, - List topicPartitions, - Coder keyCoder, - Coder valueCoder, - @Nullable SerializableFunction, Instant> timestampFn, - @Nullable SerializableFunction, Instant> watermarkFn, - SerializableFunction, Consumer> consumerFactoryFn, - Map consumerConfig, - long maxNumRecords, - @Nullable Duration maxReadTime) { - - this.topics = topics; - this.topicPartitions = topicPartitions; - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; - this.timestampFn = timestampFn; - this.watermarkFn = watermarkFn; - this.consumerFactoryFn = consumerFactoryFn; - this.consumerConfig = consumerConfig; - this.maxNumRecords = maxNumRecords; - this.maxReadTime = maxReadTime; - } + public static class Read extends TypedRead { /** * Returns a new {@link Read} with Kafka consumer pointing to {@code bootstrapServers}. @@ -244,7 +163,7 @@ public Read withTopics(List topics) { checkState(topicPartitions.isEmpty(), "Only topics or topicPartitions can be set, not both"); return new Read(ImmutableList.copyOf(topics), topicPartitions, keyCoder, valueCoder, - timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** @@ -257,54 +176,25 @@ public Read withTopicPartitions(List topicPartitions) { checkState(topics.isEmpty(), "Only topics or topicPartitions can be set, not both"); return new Read(topics, ImmutableList.copyOf(topicPartitions), keyCoder, valueCoder, - timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** * Returns a new {@link Read} with {@link Coder} for key bytes. - *

Since this changes the type for key, settings that depend on the type - * ({@link #withTimestampFn(SerializableFunction)} and - * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. */ public Read withKeyCoder(Coder keyCoder) { - checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); - checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); - return new Read(topics, topicPartitions, keyCoder, valueCoder, null, null, + return new Read(topics, topicPartitions, keyCoder, valueCoder, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** * Returns a new {@link Read} with {@link Coder} for value bytes. - *

Since this changes the type for key, settings that depend on the type - * ({@link #withTimestampFn(SerializableFunction)} and - * {@link #withWatermarkFn(SerializableFunction)}) make sense only after the coders are set. */ public Read withValueCoder(Coder valueCoder) { - checkState(timestampFn == null, "Set timestampFn after setting key and value coders"); - checkState(watermarkFn == null, "Set watermarkFn after setting key and value coders"); - return new Read(topics, topicPartitions, keyCoder, valueCoder, null, null, + return new Read(topics, topicPartitions, keyCoder, valueCoder, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } - /** - * A function to assign a timestamp to a record. Default is processing timestamp. - */ - public Read withTimestampFn(SerializableFunction, Instant> timestampFn) { - checkNotNull(timestampFn); - return new Read(topics, topicPartitions, keyCoder, valueCoder, - timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); - } - - /** - * A function to calculate watermark after a record. Default is last record timestamp - * @see #withTimestampFn(SerializableFunction) - */ - public Read withWatermarkFn(SerializableFunction, Instant> watermarkFn) { - checkNotNull(watermarkFn); - return new Read(topics, topicPartitions, keyCoder, valueCoder, - timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); - } - /** * A factory to create Kafka {@link Consumer} from consumer configuration. * This is useful for supporting another version of Kafka consumer. @@ -313,7 +203,7 @@ public Read withWatermarkFn(SerializableFunction, Instant> waterm public Read withConsumerFactoryFn( SerializableFunction, Consumer> consumerFactoryFn) { return new Read(topics, topicPartitions, keyCoder, valueCoder, - timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** @@ -328,8 +218,8 @@ public Read updateConsumerProperties(Map configUpdates) { Map config = new HashMap<>(consumerConfig); config.putAll(configUpdates); - return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, - watermarkFn, consumerFactoryFn, config, maxNumRecords, maxReadTime); + return new Read(topics, topicPartitions, keyCoder, valueCoder, + consumerFactoryFn, config, maxNumRecords, maxReadTime); } /** @@ -337,8 +227,8 @@ public Read updateConsumerProperties(Map configUpdates) { * Mainly used for tests and demo applications. */ public Read withMaxNumRecords(long maxNumRecords) { - return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, - watermarkFn, consumerFactoryFn, consumerConfig, maxNumRecords, null); + return new Read(topics, topicPartitions, keyCoder, valueCoder, + consumerFactoryFn, consumerConfig, maxNumRecords, null); } /** @@ -348,62 +238,127 @@ public Read withMaxNumRecords(long maxNumRecords) { * applications. */ public Read withMaxReadTime(Duration maxReadTime) { - return new Read(topics, topicPartitions, keyCoder, valueCoder, timestampFn, - watermarkFn, consumerFactoryFn, consumerConfig, Long.MAX_VALUE, maxReadTime); + return new Read(topics, topicPartitions, keyCoder, valueCoder, + consumerFactoryFn, consumerConfig, Long.MAX_VALUE, maxReadTime); + } + + /////////////////////////////////////////////////////////////////////////////////////// + + private Read( + List topics, + List topicPartitions, + Coder keyCoder, + Coder valueCoder, + SerializableFunction, Consumer> consumerFactoryFn, + Map consumerConfig, + long maxNumRecords, + @Nullable Duration maxReadTime) { + + super(topics, topicPartitions, keyCoder, valueCoder, null, null, + consumerFactoryFn, consumerConfig, maxNumRecords, maxReadTime); } /** - * A read transform that includes Kafka metadata along with key and value. - * @see KafkaRecord + * A set of properties that are not required or don't make sense for our consumer. */ - public ReadWithMetadata withMetadata() { - return new ReadWithMetadata(this, - timestampFn != null ? unwrapKafkaAndThen(timestampFn) : null, - watermarkFn != null ? unwrapKafkaAndThen(watermarkFn) : null); - } + private static final Map ignoredConsumerProperties = ImmutableMap.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDecoderFn instead" + // "group.id", "enable.auto.commit", "auto.commit.interval.ms" : + // lets allow these, applications can have better resume point for restarts. + ); - // utility method to convert KafkRecord to user KV before applying user functions - private static SerializableFunction, OutT> - unwrapKafkaAndThen(final SerializableFunction, OutT> fn) { - return new SerializableFunction, OutT>() { - public OutT apply(KafkaRecord record) { - return fn.apply(record.getKV()); + // set config defaults + private static final Map DEFAULT_CONSUMER_PROPERTIES = + ImmutableMap.of( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName(), + + // Use large receive buffer. Once KAFKA-3135 is fixed, this _may_ not be required. + // with default value of of 32K, It takes multiple seconds between successful polls. + // All the consumer work is done inside poll(), with smaller send buffer size, it + // takes many polls before a 1MB chunk from the server is fully read. In my testing + // about half of the time select() inside kafka consumer waited for 20-30ms, though + // the server had lots of data in tcp send buffers on its side. Compared to default, + // this setting increased throughput increased by many fold (3-4x). + ConsumerConfig.RECEIVE_BUFFER_CONFIG, 512 * 1024, + + // default to latest offset when we are not resuming. + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest", + // disable auto commit of offsets. we don't require group_id. could be enabled by user. + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + + // default Kafka 0.9 Consumer supplier. + private static final SerializableFunction, Consumer> + KAFKA_9_CONSUMER_FACTORY_FN = + new SerializableFunction, Consumer>() { + public Consumer apply(Map config) { + return new KafkaConsumer<>(config); } }; - } + } + + /** + * A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more + * information on usage and configuration. + */ + public static class TypedRead + extends PTransform>> { /** - * Creates an {@link UnboundedSource, ?>} with the configuration in - * {@link ReadWithMetadata}. Primary use case is unit tests, should not be used in an - * application. + * A function to assign a timestamp to a record. Default is processing timestamp. */ - @VisibleForTesting - UnboundedKafkaSource> makeSource() { - return new UnboundedKafkaSource>( - -1, - topics, - topicPartitions, - keyCoder, - valueCoder, - KvCoder.of(keyCoder, valueCoder), - unwrapKafkaAndThen(new IdentityFn>()), - timestampFn == null ? null : unwrapKafkaAndThen(timestampFn), - Optional.fromNullable(watermarkFn == null ? null : unwrapKafkaAndThen(watermarkFn)), - consumerFactoryFn, - consumerConfig); + public TypedRead withTimestampFn2( + SerializableFunction, Instant> timestampFn) { + checkNotNull(timestampFn); + return new TypedRead(topics, topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, + maxNumRecords, maxReadTime); } - @Override - public PCollection> apply(PInput input) { - return applyHelper(input, makeSource()); + /** + * A function to calculate watermark after a record. Default is last record timestamp + * @see #withTimestampFn(SerializableFunction) + */ + public TypedRead withWatermarkFn2( + SerializableFunction, Instant> watermarkFn) { + checkNotNull(watermarkFn); + return new TypedRead(topics, topicPartitions, keyCoder, valueCoder, + timestampFn, watermarkFn, consumerFactoryFn, consumerConfig, + maxNumRecords, maxReadTime); } /** - * Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. + * A function to assign a timestamp to a record. Default is processing timestamp. */ - PCollection applyHelper(PInput input, UnboundedSource source) { - Unbounded unbounded = com.google.cloud.dataflow.sdk.io.Read.from(source); - PTransform> transform = unbounded; + public TypedRead withTimestampFn(SerializableFunction, Instant> timestampFn) { + checkNotNull(timestampFn); + return withTimestampFn2(unwrapKafkaAndThen(timestampFn)); + } + + /** + * A function to calculate watermark after a record. Default is last record timestamp + * @see #withTimestampFn(SerializableFunction) + */ + public TypedRead withWatermarkFn(SerializableFunction, Instant> watermarkFn) { + checkNotNull(watermarkFn); + return withWatermarkFn2(unwrapKafkaAndThen(watermarkFn)); + } + + /** + * Returns a {@link PTransform} for PCollection of {@link KV}, dropping Kafka metatdata. + */ + public PTransform>> withoutMetadata() { + return new TypedWithoutMetadata(this); + } + + @Override + public PCollection> apply(PBegin input) { + // Handles unbounded source to bounded conversion if maxNumRecords or maxReadTime is set. + Unbounded> unbounded = + com.google.cloud.dataflow.sdk.io.Read.from(makeSource()); + + PTransform>> transform = unbounded; if (maxNumRecords < Long.MAX_VALUE) { transform = unbounded.withMaxNumRecords(maxNumRecords); @@ -413,93 +368,114 @@ PCollection applyHelper(PInput input, UnboundedSource source) { return input.getPipeline().apply(transform); } - } - /** - * Similar to {@link Read}, except that the transform returns a PCollection of - * {@link KafkaRecord} which includes Kafka metadata : topic name, partition, offset. - */ - public static class ReadWithMetadata - extends PTransform>> { + //////////////////////////////////////////////////////////////////////////////////////// - private final Read kvRead; - @Nullable private final SerializableFunction, Instant> timestampFn; - @Nullable private final SerializableFunction, Instant> watermarkFn; + protected final List topics; + protected final List topicPartitions; // mutually exclusive with topics + protected final Coder keyCoder; + protected final Coder valueCoder; + @Nullable protected final SerializableFunction, Instant> timestampFn; + @Nullable protected final SerializableFunction, Instant> watermarkFn; + protected final + SerializableFunction, Consumer> consumerFactoryFn; + protected final Map consumerConfig; + protected final long maxNumRecords; // bounded read, mainly for testing + protected final Duration maxReadTime; // bounded read, mainly for testing - ReadWithMetadata( - Read kvRead, + private TypedRead(List topics, + List topicPartitions, + Coder keyCoder, + Coder valueCoder, @Nullable SerializableFunction, Instant> timestampFn, - @Nullable SerializableFunction, Instant> watermarkFn) { + @Nullable SerializableFunction, Instant> watermarkFn, + SerializableFunction, Consumer> consumerFactoryFn, + Map consumerConfig, + long maxNumRecords, + @Nullable Duration maxReadTime) { + super("KafkaIO.Read"); - this.kvRead = kvRead; + this.topics = topics; + this.topicPartitions = topicPartitions; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; this.timestampFn = timestampFn; this.watermarkFn = watermarkFn; - } - - // Interface Note: - // Instead of repeating many of the builder methods ('withTopics()' etc) in Reader, we expect - // the user to set all those before Reader.withMetaData(). we still need to let users - // override timestamp functions (in cases where these functions need metadata). - - /** - * A function to assign a timestamp to a record. Default is processing timestamp. - */ - public ReadWithMetadata withTimestampFn( - SerializableFunction, Instant> timestampFn) { - checkNotNull(timestampFn); - return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); - } - - /** - * A function to calculate watermark after a record. Default is last record timestamp - * @see #withTimestampFn(SerializableFunction) - */ - public ReadWithMetadata withWatermarkFn( - SerializableFunction, Instant> watermarkFn) { - checkNotNull(watermarkFn); - return new ReadWithMetadata(kvRead, timestampFn, watermarkFn); + this.consumerFactoryFn = consumerFactoryFn; + this.consumerConfig = consumerConfig; + this.maxNumRecords = maxNumRecords; + this.maxReadTime = maxReadTime; } /** * Creates an {@link UnboundedSource, ?>} with the configuration in - * {@link ReadWithMetadata}. Primary use case is unit tests, should not be used in an + * {@link TypedRead}. Primary use case is unit tests, should not be used in an * application. */ @VisibleForTesting - UnboundedKafkaSource> makeSource() { - return new UnboundedKafkaSource<>( + UnboundedSource, KafkaCheckpointMark> makeSource() { + return new UnboundedKafkaSource( -1, - kvRead.topics, - kvRead.topicPartitions, - kvRead.keyCoder, - kvRead.valueCoder, - KafkaRecordCoder.of(kvRead.keyCoder, kvRead.valueCoder), - new IdentityFn>(), + topics, + topicPartitions, + keyCoder, + valueCoder, timestampFn, Optional.fromNullable(watermarkFn), - kvRead.consumerFactoryFn, - kvRead.consumerConfig); + consumerFactoryFn, + consumerConfig); + } + + // utility method to convert KafkRecord to user KV before applying user functions + private static SerializableFunction, OutT> + unwrapKafkaAndThen(final SerializableFunction, OutT> fn) { + return new SerializableFunction, OutT>() { + public OutT apply(KafkaRecord record) { + return fn.apply(record.getKV()); + } + }; + } + } + + /** + * A {@link PTransform} to read from Kafka topics. Similar to {@link KafkaIO.Typed}, but removes + * Kafka metatdata and returns a {@link PCollection} of {@link KV}. + * See {@link KafkaIO} for more information on usage and configuration of reader. + */ + public static class TypedWithoutMetadata extends PTransform>> { + + private final TypedRead typedRead; + + TypedWithoutMetadata(TypedRead read) { + super("KafkaIO.Read"); + this.typedRead = read; } @Override - public PCollection> apply(PInput input) { - return kvRead.applyHelper(input, makeSource()); + public PCollection> apply(PBegin begin) { + return typedRead + .apply(begin) + .apply("Remove Kafka Metadata", + ParDo.of(new DoFn, KV>() { + @Override + public void processElement(ProcessContext ctx) { + ctx.output(ctx.element().getKV()); + } + })); } } /** Static class, prevent instantiation. */ private KafkaIO() {} - private static class UnboundedKafkaSource - extends UnboundedSource { + private static class UnboundedKafkaSource + extends UnboundedSource, KafkaCheckpointMark> { private final int id; // split id, mainly for debugging private final List topics; private final List assignedPartitions; private final Coder keyCoder; private final Coder valueCoder; - private final Coder defaultOutputCoder; - private final SerializableFunction, T> converterFn; // covert to userTuype. private final SerializableFunction, Instant> timestampFn; // would it be a good idea to pass currentTimestamp to watermarkFn? private final Optional, Instant>> watermarkFn; @@ -513,8 +489,6 @@ public UnboundedKafkaSource( List assignedPartitions, Coder keyCoder, Coder valueCoder, - Coder defaultOutputCoder, - SerializableFunction, T> converterFn, @Nullable SerializableFunction, Instant> timestampFn, Optional, Instant>> watermarkFn, SerializableFunction, Consumer> consumerFactoryFn, @@ -525,8 +499,6 @@ public UnboundedKafkaSource( this.topics = topics; this.keyCoder = keyCoder; this.valueCoder = valueCoder; - this.defaultOutputCoder = defaultOutputCoder; - this.converterFn = converterFn; this.timestampFn = (timestampFn == null ? new NowTimestampFn>() : timestampFn); this.watermarkFn = watermarkFn; @@ -544,7 +516,7 @@ public UnboundedKafkaSource( * {@code } and then assigned to splits in round-robin order. */ @Override - public List> generateInitialSplits( + public List> generateInitialSplits( int desiredNumSplits, PipelineOptions options) throws Exception { List partitions = new ArrayList<>(assignedPartitions); @@ -587,7 +559,7 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { assignments.get(i % numSplits).add(partitions.get(i)); } - List> result = new ArrayList<>(numSplits); + List> result = new ArrayList<>(numSplits); for (int i = 0; i < numSplits; i++) { List assignedToSplit = assignments.get(i); @@ -595,14 +567,12 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { LOG.info("Partitions assigned to split {} (total {}): {}", i, assignedToSplit.size(), Joiner.on(",").join(assignedToSplit)); - result.add(new UnboundedKafkaSource( + result.add(new UnboundedKafkaSource( i, this.topics, assignedToSplit, this.keyCoder, this.valueCoder, - this.defaultOutputCoder, - this.converterFn, this.timestampFn, this.watermarkFn, this.consumerFactoryFn, @@ -613,19 +583,19 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { } @Override - public UnboundedKafkaReader createReader(PipelineOptions options, + public UnboundedKafkaReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { if (assignedPartitions.isEmpty()) { LOG.warn("hack: working around DirectRunner issue. It does not generateSplits()"); // generate single split and return reader from it. try { - return new UnboundedKafkaReader( + return new UnboundedKafkaReader( generateInitialSplits(1, options).get(0), checkpointMark); } catch (Exception e) { Throwables.propagate(e); } } - return new UnboundedKafkaReader(this, checkpointMark); + return new UnboundedKafkaReader(this, checkpointMark); } @Override @@ -647,14 +617,14 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { - return defaultOutputCoder; + public Coder> getDefaultOutputCoder() { + return KafkaRecordCoder.of(keyCoder, valueCoder); } } - private static class UnboundedKafkaReader extends UnboundedReader { + private static class UnboundedKafkaReader extends UnboundedReader> { - private final UnboundedKafkaSource source; + private final UnboundedKafkaSource source; private final String name; private Consumer consumer; private final List partitionStates; @@ -739,7 +709,7 @@ synchronized long approxBacklogInBytes() { } public UnboundedKafkaReader( - UnboundedKafkaSource source, + UnboundedKafkaSource source, @Nullable KafkaCheckpointMark checkpointMark) { this.source = source; @@ -906,15 +876,19 @@ public boolean advance() throws IOException { LOG.info("{} : first record offset {}", name, offset); } + curRecord = null; // user coders below might throw. + // apply user coders. might want to allow skipping records that fail to decode. - curRecord = new KafkaRecord( + // TODO: wrap exceptions from coders to make explicit to users + KafkaRecord record = new KafkaRecord( rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), decode(rawRecord.key(), source.keyCoder), decode(rawRecord.value(), source.valueCoder)); - curTimestamp = source.timestampFn.apply(curRecord); + curTimestamp = source.timestampFn.apply(record); + curRecord = record; int recordSize = (rawRecord.key() == null ? 0 : rawRecord.key().length) + (rawRecord.value() == null ? 0 : rawRecord.value().length); @@ -941,6 +915,7 @@ private static T decode(byte[] bytes, Coder coder) throws IOException { return coder.decode(new ExposedByteArrayInputStream(toDecode), Coder.Context.OUTER); } + // update latest offset for each partition. // called from offsetFetcher thread private void updateLatestOffsets() { for (PartitionState p : partitionStates) { @@ -984,14 +959,14 @@ public PartitionMark apply(PartitionState p) { } @Override - public UnboundedSource getCurrentSource() { + public UnboundedSource, ?> getCurrentSource() { return source; } @Override - public T getCurrent() throws NoSuchElementException { + public KafkaRecord getCurrent() throws NoSuchElementException { // should we delay updating consumed offset till this point? Mostly not required. - return source.converterFn.apply(curRecord); + return curRecord; } @Override diff --git a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java index bb80f61eac..b4bdc83ce9 100644 --- a/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java +++ b/contrib/kafka/src/test/java/com/google/cloud/dataflow/contrib/kafka/KafkaIOTest.java @@ -159,7 +159,7 @@ public Consumer apply(Map config) { * Creates a consumer with two topics, with 5 partitions each. * numElements are (round-robin) assigned all the 10 partitions. */ - private static UnboundedSource, KafkaCheckpointMark> mkKafkaSource( + private static KafkaIO.TypedRead mkKafkaReadTransform( int numElements, @Nullable SerializableFunction, Instant> timestampFn) { @@ -169,13 +169,14 @@ private static UnboundedSource, KafkaCheckpointMark> mkKafkaSou .withBootstrapServers("none") .withTopics(topics) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 20 partitions - .withValueCoder(BigEndianLongCoder.of()); + .withValueCoder(BigEndianLongCoder.of()) + .withMaxNumRecords(numElements); if (timestampFn != null) { - reader = reader.withTimestampFn(timestampFn); + return reader.withTimestampFn(timestampFn); + } else { + return reader; } - - return reader.makeSource(); } private static class AssertMultipleOf implements SerializableFunction, Void> { @@ -221,8 +222,8 @@ public void testUnboundedSource() { int numElements = 1000; PCollection input = p - .apply(Read.from(mkKafkaSource(numElements, new ValueAsTimestampFn())) - .withMaxNumRecords(numElements)) + .apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()) + .withoutMetadata()) .apply(Values.create()); addCountingAsserts(input, numElements); @@ -237,7 +238,7 @@ public void testUnboundedSourceWithExplicitPartitions() { List topics = ImmutableList.of("test"); - KafkaIO.Read reader = KafkaIO.read() + KafkaIO.TypedRead reader = KafkaIO.read() .withBootstrapServers("none") .withTopicPartitions(ImmutableList.of(new TopicPartition("test", 5))) .withConsumerFactoryFn(new ConsumerFactoryFn(topics, 10, numElements)) // 10 partitions @@ -245,7 +246,7 @@ public void testUnboundedSourceWithExplicitPartitions() { .withMaxNumRecords(numElements / 10); PCollection input = p - .apply(reader) + .apply(reader.withoutMetadata()) .apply(Values.create()); // assert that every element is a multiple of 5. @@ -274,8 +275,7 @@ public void testUnboundedSourceTimestamps() { int numElements = 1000; PCollection input = p - .apply(Read.from(mkKafkaSource(numElements, new ValueAsTimestampFn())) - .withMaxNumRecords(numElements)) + .apply(mkKafkaReadTransform(numElements, new ValueAsTimestampFn()).withoutMetadata()) .apply(Values.create()); addCountingAsserts(input, numElements); @@ -289,6 +289,13 @@ public void testUnboundedSourceTimestamps() { p.run(); } + private static class RemoveKafkaMetadata extends DoFn, KV> { + @Override + public void processElement(ProcessContext ctx) throws Exception { + ctx.output(ctx.element().getKV()); + } + } + @Test @Category(RunnableOnService.class) public void testUnboundedSourceSplits() throws Exception { @@ -296,8 +303,9 @@ public void testUnboundedSourceSplits() throws Exception { int numElements = 1000; int numSplits = 10; - UnboundedSource, ?> initial = mkKafkaSource(numElements, null); - List, ?>> splits = + UnboundedSource, ?> initial = + mkKafkaReadTransform(numElements, null).makeSource(); + List, ?>> splits = initial.generateInitialSplits(numSplits, p.getOptions()); assertEquals("Expected exact splitting", numSplits, splits.size()); @@ -307,6 +315,7 @@ public void testUnboundedSourceSplits() throws Exception { for (int i = 0; i < splits.size(); ++i) { pcollections = pcollections.and( p.apply("split" + i, Read.from(splits.get(i)).withMaxNumRecords(elementsPerSplit)) + .apply("Remove Metadata " + i, ParDo.of(new RemoveKafkaMetadata())) .apply("collection " + i, Values.create())); } PCollection input = pcollections.apply(Flatten.pCollections()); @@ -331,23 +340,25 @@ public void testUnboundedSourceCheckpointMark() throws Exception { int numElements = 85; // 85 to make sure some partitions have more records than other. // create a single split: - UnboundedSource, KafkaCheckpointMark> source = - mkKafkaSource(numElements, new ValueAsTimestampFn()) + UnboundedSource, KafkaCheckpointMark> source = + mkKafkaReadTransform(numElements, new ValueAsTimestampFn()) + .makeSource() .generateInitialSplits(1, PipelineOptionsFactory.fromArgs(new String[0]).create()) .get(0); - UnboundedReader> reader = source.createReader(null, null); + UnboundedReader> reader = source.createReader(null, null); final int numToSkip = 3; // advance once: assertTrue(reader.start()); // Advance the source numToSkip-1 elements and manually save state. for (long l = 0; l < numToSkip - 1; ++l) { - reader.advance(); + assertTrue(reader.advance()); } // Confirm that we get the expected element in sequence before checkpointing. - assertEquals(numToSkip - 1, (long) reader.getCurrent().getValue()); + + assertEquals(numToSkip - 1, (long) reader.getCurrent().getKV().getValue()); assertEquals(numToSkip - 1, reader.getCurrentTimestamp().getMillis()); // Checkpoint and restart, and confirm that the source continues correctly. @@ -359,7 +370,7 @@ public void testUnboundedSourceCheckpointMark() throws Exception { // Confirm that we get the next elements in sequence. // This also confirms that Reader interleaves records from each partitions by the reader. for (int i = numToSkip; i < numElements; i++) { - assertEquals(i, (long) reader.getCurrent().getValue()); + assertEquals(i, (long) reader.getCurrent().getKV().getValue()); assertEquals(i, reader.getCurrentTimestamp().getMillis()); reader.advance(); } From 3d560732adfb56d572740985bd4bc3df7900b1e3 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 4 Apr 2016 17:36:19 -0700 Subject: [PATCH 69/72] KafkaIO JavaDoc --- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 76 ++++++++++++++----- .../dataflow/sdk/io/bigtable/BigtableIO.java | 2 +- 2 files changed, 60 insertions(+), 18 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 5d3c503ad0..6b0b9ae814 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -84,28 +84,70 @@ import javax.annotation.Nullable; /** - * Dataflow support for consuming Kafka topics as {@link Unbounded} sources. + * An unbounded source for Kafka topics. Kafka version 0.9 + * and above are supported. * - *

- * The Kafka partitions - * are evenly distributed among splits (workers). Dataflow checkpointing is fully supported and + *

Reading from Kafka topics

+ * + *

KafkaIO source returns unbounded collection of Kafka records as + * {@code PCollection>}. A {@link KafkaRecord} includes basic + * metadata like topic-partition and offset, along with key and value associated with a Kafka + * record. + * + *

Although most applications consumer single topic, the source can be configured to consume + * multiple topics or even a specific set of {@link TopicPartition}s. + * + *

To configure a Kafka source, you must specify at the minimum Kafka bootstrapServers + * and one or more topics to consume. The following example illustrates various options for + * configuring the source : + * + *

{@code
+ *
+ *  pipeline
+ *    .apply(KafkaIO.read()
+ *       .withBootstrapServers("broker_1:9092,broker_2:9092")
+ *       .withTopics(ImmutableList.of("topic_a", "topic_b"))
+ *       // above two are required configuration. returns PCollection
+ *
+ *       // rest of the settings are optional :
+ *
+ *       // set a Coder for Key and Value (note the change to return type)
+ *       .withKeyCoder(BigEndianLongCoder.of()) // PCollection
+ *       .withValueCoder(StringUtf8Coder.of())  // PCollection
+ *
+ *       // you can further customize KafkaConsumer used to read the records by adding more
+ *       // settings for ConsumerConfig. e.g :
+ *       .updateConsumerProperties(ImmutableMap.of("receive.buffer.bytes", 1024 * 1024))
+ *
+ *       // custom function for calculating record timestamp (default is processing time)
+ *       .withTimestampFn(new MyTypestampFunction())
+ *
+ *       // custom function for watermark (default is record timestamp)
+ *       .withWatermarkFn(new MyWatermarkFunction())
+ *
+ *       // finally, if you don't need Kafka metadata, you can drop it
+ *       .withoutMetadata() // PCollection>
+ *    )
+ *    .apply(Values.create()) // PCollection
+ *     ...
+ * }
+ * + *

Partition Assignment and Checkpointing

+ * The Kafka partitions are evenly distributed among splits (workers). + * Dataflow checkpointing is fully supported and * each split can resume from previous checkpoint. See * {@link UnboundedKafkaSource#generateInitialSplits(int, PipelineOptions)} for more details on * splits and checkpoint support. * - *
{@code
- * Usage:
- *   pipeline
- *     .apply("read_topic_a_and_b",
- *        KafkaIO.read()
- *         .withBootstrapServers("broker_1:9092,broker_2:9092")
- *         .withTopics(ImmutableList.of("topic_a", "topic_b")) // or withTopicPartitions(List<>)
- *         .withValueCoder(StringUtf8Coder.of())
- *         .withTimestampFn(timestampFn) // optional
- *         .withWatermarkFn(watermarkFn)) // optional
- *     .apply(Values.create()) // discard keys
- *     ...
- *}
+ *

When the pipeline starts for the first time without any checkpoint, the source starts + * consuming from the latest offsets. You can override this behavior to consume from the + * beginning by setting appropriate appropriate properties in {@link ConsumerConfig}, through + * {@link Read#updateConsumerProperties(Map)}. + * + *

Advanced Kafka Configuration

+ * KafakIO allows setting most of the properties in {@link ConsumerConfig}. E.g. if you would like + * to enable offset auto commit (for external monitoring or other purposes), you can set + * "group.id", "enable.auto.commit", etc. */ public class KafkaIO { diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java index 562d253d77..81a85fad37 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java @@ -72,7 +72,7 @@ *

Reading from Cloud Bigtable

* *

The Bigtable source returns a set of rows from a single table, returning a - * {@code PCollection<Row>}. + * {@code PCollection}. * *

To configure a Cloud Bigtable source, you must supply a table id and a {@link BigtableOptions} * or builder configured with the project and other information necessary to identify the From 5e3b54eaafdeb5b02695f26bdabbf81fbdaf02d0 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 5 Apr 2016 11:57:39 -0700 Subject: [PATCH 70/72] review comments --- .../kafka/examples/TopHashtagsExample.java | 6 +-- .../cloud/dataflow/contrib/kafka/KafkaIO.java | 45 +++++++++++-------- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java index b6b49c9d15..2a575f1d8b 100644 --- a/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java +++ b/contrib/examples/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/examples/TopHashtagsExample.java @@ -234,7 +234,7 @@ private static class KafkaWriter extends DoFn { private final String topic; private final Map config; - private transient KafkaProducer producer = null; + private static transient KafkaProducer producer = null; public KafkaWriter(Options options) { this.topic = options.getOutputTopic(); @@ -253,12 +253,12 @@ public void startBundle(Context c) throws Exception { @Override public void finishBundle(Context c) throws Exception { - producer.close(); + producer.flush(); } @Override public void processElement(ProcessContext ctx) throws Exception { - LOG.info("Top Hashtags : {}", ctx.element()); + LOG.trace("Top Hashtags : {}", ctx.element()); producer.send(new ProducerRecord(topic, ctx.element())); } } diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 6b0b9ae814..477586dcaa 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import com.google.api.client.repackaged.com.google.common.annotations.VisibleForTesting; import com.google.cloud.dataflow.contrib.kafka.KafkaCheckpointMark.PartitionMark; import com.google.cloud.dataflow.sdk.coders.ByteArrayCoder; import com.google.cloud.dataflow.sdk.coders.Coder; @@ -39,6 +38,7 @@ import com.google.cloud.dataflow.sdk.values.PBegin; import com.google.cloud.dataflow.sdk.values.PCollection; import com.google.cloud.dataflow.sdk.values.PInput; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; @@ -628,8 +628,7 @@ public int compare(TopicPartition tp1, TopicPartition tp2) { public UnboundedKafkaReader createReader(PipelineOptions options, KafkaCheckpointMark checkpointMark) { if (assignedPartitions.isEmpty()) { - LOG.warn("hack: working around DirectRunner issue. It does not generateSplits()"); - // generate single split and return reader from it. + LOG.warn("Looks like generateSplits() is not called. Generate single split."); try { return new UnboundedKafkaReader( generateInitialSplits(1, options).get(0), checkpointMark); @@ -647,6 +646,8 @@ public Coder getCheckpointMarkCoder() { @Override public boolean requiresDeduping() { + // Kafka records are ordered with in partitions. In addition checkpoint guarantees + // records are not consumed twice. return false; } @@ -674,6 +675,10 @@ private static class UnboundedKafkaReader extends UnboundedReader curBatch = Collections.emptyIterator(); + private static final Duration KAFKA_POLL_TIMEOUT = Duration.millis(1000); + // how long to wait for new records from kafka consumer inside advance() + private static final Duration NEW_RECORDS_POLL_TIMEOUT = Duration.millis(10); + // Use a separate thread to read Kafka messages. Kafka Consumer does all its work including // network I/O inside poll(). Polling only inside #advance(), especially with a small timeout // like 100 milliseconds does not work well. This along with large receive buffer for @@ -777,8 +782,8 @@ public PartitionState apply(TopicPartition tp) { TopicPartition assigned = source.assignedPartitions.get(i); checkState(ckptMark.getTopicPartition().equals(assigned), - "checkpointed partition %s and assinged partition %s don't match at position %d", - ckptMark.getTopicPartition(), assigned, i); + "checkpointed partition %s and assigned partition %s don't match", + ckptMark.getTopicPartition(), assigned); partitionStates.get(i).consumedOffset = ckptMark.getOffset(); } @@ -789,19 +794,19 @@ private void consumerPollLoop() { // Read in a loop and enqueue the batch of records, if any, to availableRecordsQueue while (!closed) { try { - ConsumerRecords records = consumer.poll(1000); + ConsumerRecords records = consumer.poll(KAFKA_POLL_TIMEOUT.getMillis()); if (!records.isEmpty()) { availableRecordsQueue.put(records); // blocks until dequeued. } } catch (InterruptedException e) { - LOG.warn(this + " consumer thread is interrupted", e); // not expected + LOG.warn("{}: consumer thread is interrupted", this, e); // not expected break; } catch (WakeupException e) { break; } } - LOG.info("{} : Returning from consumer pool loop", this); + LOG.info("{}: Returning from consumer pool loop", this); } private void nextBatch() { @@ -809,9 +814,10 @@ private void nextBatch() { ConsumerRecords records; try { - records = availableRecordsQueue.poll(10, TimeUnit.MILLISECONDS); + records = availableRecordsQueue.poll(NEW_RECORDS_POLL_TIMEOUT.getMillis(), + TimeUnit.MILLISECONDS); } catch (InterruptedException e) { - LOG.warn("Unexpected", e); + LOG.warn("{}: Unexpected", this, e); return; } @@ -843,7 +849,7 @@ public boolean start() throws IOException { LOG.info("{}: resuming {} at {}", name, p.topicPartition, p.consumedOffset + 1); consumer.seek(p.topicPartition, p.consumedOffset + 1); } else { - LOG.info("{} : resuming {} at default offset", name, p.topicPartition); + LOG.info("{}: resuming {} at default offset", name, p.topicPartition); } } @@ -904,18 +910,19 @@ public boolean advance() throws IOException { if (consumed >= 0 && offset <= consumed) { // -- (a) // this can happen when compression is enabled in Kafka (seems to be fixed in 0.10) // should we check if the offset is way off from consumedOffset (say > 1M)? - LOG.warn("ignoring already consumed offset {} for {}", offset, pState.topicPartition); + LOG.warn("{}: ignoring already consumed offset {} for {}", + this, offset, pState.topicPartition); continue; } // sanity check if (consumed >= 0 && (offset - consumed) != 1) { - LOG.warn("gap in offsets for {} after {}. {} records missing.", - pState.topicPartition, consumed, offset - consumed - 1); + LOG.warn("{}: gap in offsets for {} after {}. {} records missing.", + this, pState.topicPartition, consumed, offset - consumed - 1); } if (curRecord == null) { - LOG.info("{} : first record offset {}", name, offset); + LOG.info("{}: first record offset {}", name, offset); } curRecord = null; // user coders below might throw. @@ -966,21 +973,21 @@ private void updateLatestOffsets() { long offset = offsetConsumer.position(p.topicPartition); p.setLatestOffset(offset);; } catch (Exception e) { - LOG.warn(this + " : exception while fetching latest offsets. ignored.", e); + LOG.warn("{}: exception while fetching latest offsets. ignored.", this, e); p.setLatestOffset(-1L); // reset } - LOG.debug("{} : latest offset update for {} : {} (consumed offset {}, avg record size {})", + LOG.debug("{}: latest offset update for {} : {} (consumed offset {}, avg record size {})", this, p.topicPartition, p.latestOffset, p.consumedOffset, p.avgRecordSize); } - LOG.debug("{} : backlog {}", this, getSplitBacklogBytes()); + LOG.debug("{}: backlog {}", this, getSplitBacklogBytes()); } @Override public Instant getWatermark() { if (curRecord == null) { - LOG.warn("{} : getWatermark() : no records have been read yet.", name); + LOG.warn("{}: getWatermark() : no records have been read yet.", name); return initialWatermark; } From 4e432bdd25d38e66e55bb41e1f38b6a2d24de333 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 5 Apr 2016 13:03:45 -0700 Subject: [PATCH 71/72] review comments --- .../com/google/cloud/dataflow/contrib/kafka/KafkaIO.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java index 477586dcaa..ad254ee735 100644 --- a/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java +++ b/contrib/kafka/src/main/java/com/google/cloud/dataflow/contrib/kafka/KafkaIO.java @@ -253,8 +253,8 @@ public Read withConsumerFactoryFn( */ public Read updateConsumerProperties(Map configUpdates) { for (String key : configUpdates.keySet()) { - checkArgument(!ignoredConsumerProperties.containsKey(key), - "No need to configure '%s'. %s", key, ignoredConsumerProperties.get(key)); + checkArgument(!IGNORED_CONSUMER_PROPERTIES.containsKey(key), + "No need to configure '%s'. %s", key, IGNORED_CONSUMER_PROPERTIES.get(key)); } Map config = new HashMap<>(consumerConfig); @@ -303,7 +303,7 @@ private Read( /** * A set of properties that are not required or don't make sense for our consumer. */ - private static final Map ignoredConsumerProperties = ImmutableMap.of( + private static final Map IGNORED_CONSUMER_PROPERTIES = ImmutableMap.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyDecoderFn instead", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueDecoderFn instead" // "group.id", "enable.auto.commit", "auto.commit.interval.ms" : From 95912d19f158fa399c0da9d35f5af700a23a68cb Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Tue, 5 Apr 2016 13:07:47 -0700 Subject: [PATCH 72/72] revert a small fix --- .../com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java index 81a85fad37..562d253d77 100644 --- a/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java +++ b/sdk/src/main/java/com/google/cloud/dataflow/sdk/io/bigtable/BigtableIO.java @@ -72,7 +72,7 @@ *

Reading from Cloud Bigtable

* *

The Bigtable source returns a set of rows from a single table, returning a - * {@code PCollection}. + * {@code PCollection<Row>}. * *

To configure a Cloud Bigtable source, you must supply a table id and a {@link BigtableOptions} * or builder configured with the project and other information necessary to identify the