diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index 7fff64199e0fd..9645d7c20083e 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -23,7 +23,10 @@
import org.apache.beam.sdk.coders.ByteArrayCoder;
import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.KvCoder;
import org.apache.beam.sdk.coders.SerializableCoder;
+import org.apache.beam.sdk.coders.VoidCoder;
import org.apache.beam.sdk.io.Read.Unbounded;
import org.apache.beam.sdk.io.UnboundedSource;
import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark;
@@ -34,10 +37,12 @@
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.CoderUtils;
import org.apache.beam.sdk.util.ExposedByteArrayInputStream;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PDone;
import org.apache.beam.sdk.values.PInput;
import com.google.common.annotations.VisibleForTesting;
@@ -56,10 +61,17 @@
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.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
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.apache.kafka.common.serialization.Serializer;
import org.joda.time.Duration;
import org.joda.time.Instant;
import org.slf4j.Logger;
@@ -86,8 +98,8 @@
import javax.annotation.Nullable;
/**
- * An unbounded source for Kafka topics. Kafka version 0.9
- * and above are supported.
+ * An unbounded source and a sink for Kafka topics.
+ * Kafka version 0.9 and above are supported.
*
*
Reading from Kafka topics
*
@@ -146,25 +158,54 @@
* beginning by setting appropriate appropriate properties in {@link ConsumerConfig}, through
* {@link Read#updateConsumerProperties(Map)}.
*
+ * Writing to Kafka
+ *
+ * KafkaIO sink supports writing key-value pairs to a Kafka topic. Users can also write
+ * just the values. To configure a Kafka sink, you must specify at the minimum Kafka
+ * bootstrapServers and the topic to write to. The following example illustrates various
+ * options for configuring the sink:
+ *
+ * {@code
+ *
+ * pipeline
+ * .apply(...) // returns PCollection>
+ * .apply(KafkaIO.write()
+ * .withBootstrapServers("broker_1:9092,broker_2:9092")
+ * .withTopic("results")
+ *
+ * // set Coder for Key and Value
+ * .withKeyCoder(BigEndianLongCoder.of())
+ * .withValueCoder(StringUtf8Coder.of())
+
+ * // you can further customize KafkaProducer used to write the records by adding more
+ * // settings for ProducerConfig. e.g, to enable compression :
+ * .updateProducerProperties(ImmutableMap.of("compression.type", "gzip"))
+ * );
+ * }
+ *
+ * Often you might want to write just values without any keys to Kafka. Use {@code values()} to
+ * write records with default empty(null) key:
+ *
+ * {@code
+ * PCollection strings = ...;
+ * strings.apply(KafkaIO.write()
+ * .withBootstrapServers("broker_1:9092,broker_2:9092")
+ * .withTopic("results")
+ * .withValueCoder(StringUtf8Coder.of()) // just need coder for value
+ * .values() // writes values to Kafka with default key
+ * );
+ * }
+ *
* 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
+ * KafakIO allows setting most of the properties in {@link ConsumerConfig} for source or in
+ * {@link ProducerConfig} for sink. 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 {
- private static final Logger LOG = LoggerFactory.getLogger(KafkaIO.class);
-
- private static class NowTimestampFn implements SerializableFunction {
- @Override
- public Instant apply(T input) {
- return Instant.now();
- }
- }
-
-
/**
- * Creates and uninitialized {@link Read} {@link PTransform}. Before use, basic Kafka
+ * Creates an 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.
@@ -181,6 +222,21 @@ public static Read read() {
null);
}
+ /**
+ * Creates an uninitialized {@link Write} {@link PTransform}. Before use, Kafka configuration
+ * should be set with {@link Write#withBootstrapServers(String)} and {@link Write#withTopic}
+ * along with {@link Coder}s for (optional) key and values.
+ */
+ public static Write write() {
+ return new Write(
+ null,
+ ByteArrayCoder.of(),
+ ByteArrayCoder.of(),
+ TypedWrite.DEFAULT_PRODUCER_PROPERTIES);
+ }
+
+ ///////////////////////// Read Support \\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\
+
/**
* A {@link PTransform} to read from Kafka topics. See {@link KafkaIO} for more
* information on usage and configuration.
@@ -253,13 +309,9 @@ public Read withConsumerFactoryFn(
* Update consumer configuration with new properties.
*/
public Read updateConsumerProperties(Map configUpdates) {
- for (String key : configUpdates.keySet()) {
- checkArgument(!IGNORED_CONSUMER_PROPERTIES.containsKey(key),
- "No need to configure '%s'. %s", key, IGNORED_CONSUMER_PROPERTIES.get(key));
- }
- Map config = new HashMap<>(consumerConfig);
- config.putAll(configUpdates);
+ Map config = updateKafkaProperties(consumerConfig,
+ IGNORED_CONSUMER_PROPERTIES, configUpdates);
return new Read(topics, topicPartitions, keyCoder, valueCoder,
consumerFactoryFn, config, maxNumRecords, maxReadTime);
@@ -305,8 +357,8 @@ private Read(
* A set of properties that are not required or don't make sense for our consumer.
*/
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"
+ ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "Set keyCoder instead",
+ ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "Set valueCoder instead"
// "group.id", "enable.auto.commit", "auto.commit.interval.ms" :
// lets allow these, applications can have better resume point for restarts.
);
@@ -508,6 +560,37 @@ public void processElement(ProcessContext ctx) {
}
}
+ ////////////////////////////////////////////////////////////////////////////////////////////////
+
+ private static final Logger LOG = LoggerFactory.getLogger(KafkaIO.class);
+
+ /**
+ * Returns a new config map which is merge of current config and updates.
+ * Verifies the updates do not includes ignored properties.
+ */
+ private static Map updateKafkaProperties(
+ Map currentConfig,
+ Map ignoredProperties,
+ Map updates) {
+
+ for (String key : updates.keySet()) {
+ checkArgument(!ignoredProperties.containsKey(key),
+ "No need to configure '%s'. %s", key, ignoredProperties.get(key));
+ }
+
+ Map config = new HashMap<>(currentConfig);
+ config.putAll(updates);
+
+ return config;
+ }
+
+ private static class NowTimestampFn implements SerializableFunction {
+ @Override
+ public Instant apply(T input) {
+ return Instant.now();
+ }
+ }
+
/** Static class, prevent instantiation. */
private KafkaIO() {}
@@ -719,7 +802,6 @@ private static class PartitionState {
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;
@@ -1073,4 +1155,315 @@ public void close() throws IOException {
Closeables.close(consumer, true);
}
}
+
+ //////////////////////// Sink Support \\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\\
+
+ /**
+ * A {@link PTransform} to write to a Kafka topic. See {@link KafkaIO} for more
+ * information on usage and configuration.
+ */
+ public static class Write extends TypedWrite {
+
+ /**
+ * Returns a new {@link Write} transform with Kafka producer pointing to
+ * {@code bootstrapServers}.
+ */
+ public Write withBootstrapServers(String bootstrapServers) {
+ return updateProducerProperties(
+ ImmutableMap.of(
+ ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers));
+ }
+
+ /**
+ * Returns a new {@link Write} transform that write to given topic.
+ */
+ public Write withTopic(String topic) {
+ return new Write(topic, keyCoder, valueCoder, producerConfig);
+ }
+
+ /**
+ * Returns a new {@link Write} with {@link Coder} for serializing key (if any) to bytes.
+ * A key is optional while writing to Kafka. Note when a key is set, its hash is used to
+ * determine partition in Kafka (see {@link ProducerRecord} for more details).
+ */
+ public Write withKeyCoder(Coder keyCoder) {
+ return new Write(topic, keyCoder, valueCoder, producerConfig);
+ }
+
+ /**
+ * Returns a new {@link Write} with {@link Coder} for serializing value to bytes.
+ */
+ public Write withValueCoder(Coder valueCoder) {
+ return new Write(topic, keyCoder, valueCoder, producerConfig);
+ }
+
+ public Write updateProducerProperties(Map configUpdates) {
+ Map config = updateKafkaProperties(producerConfig,
+ TypedWrite.IGNORED_PRODUCER_PROPERTIES, configUpdates);
+ return new Write(topic, keyCoder, valueCoder, config);
+ }
+
+ private Write(
+ String topic,
+ Coder keyCoder,
+ Coder valueCoder,
+ Map producerConfig) {
+ super(topic, keyCoder, valueCoder, producerConfig,
+ Optional., Producer>>absent());
+ }
+ }
+
+ /**
+ * A {@link PTransform} to write to a Kafka topic. See {@link KafkaIO} for more
+ * information on usage and configuration.
+ */
+ public static class TypedWrite extends PTransform>, PDone> {
+
+ /**
+ * Returns a new {@link Write} with a custom function to create Kafka producer. Primarily used
+ * for tests. Default is {@link KafkaProducer}
+ */
+ public TypedWrite withProducerFactoryFn(
+ SerializableFunction