[ 
https://issues.apache.org/jira/browse/BEAM-5798?focusedWorklogId=163718&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-163718
 ]

ASF GitHub Bot logged work on BEAM-5798:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 07/Nov/18 22:56
            Start Date: 07/Nov/18 22:56
    Worklog Time Spent: 10m 
      Work Description: lukecwik closed pull request #6776: WIP: [BEAM-5798] 
Added "withTopicFn()" to set sink topics dynamically
URL: https://github.com/apache/beam/pull/6776
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 31ba72c54ba..e311688b179 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
@@ -167,7 +167,7 @@
  * PCollection<KV<Long, String>> kvColl = ...;
  * kvColl.apply(KafkaIO.<Long, String>write()
  *      .withBootstrapServers("broker_1:9092,broker_2:9092")
- *      .withTopic("results")
+ *      .withTopic("results") // use withTopicFn(SerializableFunction fn) to 
set topics dynamically
  *
  *      .withKeySerializer(LongSerializer.class)
  *      .withValueSerializer(StringSerializer.class)
@@ -862,6 +862,9 @@ private KafkaIO() {}
     @Nullable
     abstract String getTopic();
 
+    @Nullable
+    abstract SerializableFunction<KV<K, V>, String> getTopicFn();
+
     abstract Map<String, Object> getProducerConfig();
 
     @Nullable
@@ -894,6 +897,8 @@ private KafkaIO() {}
     abstract static class Builder<K, V> {
       abstract Builder<K, V> setTopic(String topic);
 
+      abstract Builder<K, V> setTopicFn(SerializableFunction<KV<K, V>, String> 
fn);
+
       abstract Builder<K, V> setProducerConfig(Map<String, Object> 
producerConfig);
 
       abstract Builder<K, V> setProducerFactoryFn(
@@ -927,9 +932,20 @@ private KafkaIO() {}
           ImmutableMap.of(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, 
bootstrapServers));
     }
 
-    /** Sets the Kafka topic to write to. */
+    /**
+     * Sets the Kafka topic to write to. Note that this overrides any 
previously function set
+     * by {@link #withTopicFn}.
+     */
     public Write<K, V> withTopic(String topic) {
-      return toBuilder().setTopic(topic).build();
+      return toBuilder().setTopic(topic).setTopicFn(null).build();
+    }
+
+    /**
+     * Sets a custom function to define sink topic dynamically. Note that this 
overrides
+     * any previously set topic by {@link #withTopic}.
+     */
+    public Write<K, V> withTopicFn(SerializableFunction<KV<K, V>, String> 
topicFn) {
+      return toBuilder().setTopic(null).setTopicFn(topicFn).build();
     }
 
     /**
@@ -1057,11 +1073,16 @@ public PDone expand(PCollection<KV<K, V>> input) {
       checkArgument(
           getProducerConfig().get(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) != 
null,
           "withBootstrapServers() is required");
-      checkArgument(getTopic() != null, "withTopic() is required");
+      checkArgument(
+          getTopic() != null || getTopicFn() != null, "withTopic() or 
withTopicFn() is required");
+
       checkArgument(getKeySerializer() != null, "withKeySerializer() is 
required");
       checkArgument(getValueSerializer() != null, "withValueSerializer() is 
required");
 
       if (isEOS()) {
+        checkArgument(getTopic() != null, "withTopic() is required with EOS 
sink");
+        checkArgument(getTopicFn() == null, "withTopicFn() can't be used 
together with EOS sink");
+
         KafkaExactlyOnceSink.ensureEOSSupport();
 
         // TODO: Verify that the group_id does not have existing state stored 
on Kafka unless
diff --git 
a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
 
b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
index beaa9a22053..3b55652a720 100644
--- 
a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
+++ 
b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
@@ -61,8 +61,9 @@ public void processElement(ProcessContext ctx) throws 
Exception {
             ? spec.getPublishTimestampFunction().getTimestamp(kv, 
ctx.timestamp()).getMillis()
             : null;
 
+    String topic = spec.getTopicFn() != null ? spec.getTopicFn().apply(kv) : 
spec.getTopic();
     producer.send(
-        new ProducerRecord<>(spec.getTopic(), null, timestampMillis, 
kv.getKey(), kv.getValue()),
+        new ProducerRecord<>(topic, null, timestampMillis, kv.getKey(), 
kv.getValue()),
         new SendCallback());
 
     elementsWritten.inc();
diff --git 
a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
 
b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
index 7d0e766e67e..8cce6ba7980 100644
--- 
a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
+++ 
b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
@@ -1034,6 +1034,41 @@ public void testSink() throws Exception {
     }
   }
 
+  @Test
+  public void testSinkDynamicTopics() throws Exception {
+    // Simply read from kafka source and write to two kafka sinks using custom 
topic function.
+    // Then verify the records are correctly published to mock kafka producer 
and into proper topic.
+
+    int numElements = 1000;
+
+    try (MockProducerWrapper producerWrapper = new MockProducerWrapper()) {
+
+      ProducerSendCompletionThread completionThread =
+          new 
ProducerSendCompletionThread(producerWrapper.mockProducer).start();
+
+      p.apply(mkKafkaReadTransform(numElements, new 
ValueAsTimestampFn()).withoutMetadata())
+          .apply(
+              KafkaIO.<Integer, Long>write()
+                  .withBootstrapServers("none")
+                  .withTopicFn(
+                      (SerializableFunction<KV<Integer, Long>, String>)
+                          input -> "topic-" + (input.getValue() < numElements 
/ 2 ? "1" : "2"))
+                  .withKeySerializer(IntegerSerializer.class)
+                  .withValueSerializer(LongSerializer.class)
+                  .withInputTimestamp()
+                  .withProducerFactoryFn(new 
ProducerFactoryFn(producerWrapper.producerKey)));
+
+      p.run();
+
+      completionThread.shutdown();
+
+      verifyProducerRecords(
+          producerWrapper.mockProducer, "topic-1", 0, numElements / 2, false, 
true);
+      verifyProducerRecords(
+          producerWrapper.mockProducer, "topic-2", numElements / 2, 
numElements, false, true);
+    }
+  }
+
   @Test
   public void testValuesSink() throws Exception {
     // similar to testSink(), but use values()' interface.
@@ -1376,6 +1411,16 @@ private static void verifyProducerRecords(
       int numElements,
       boolean keyIsAbsent,
       boolean verifyTimestamp) {
+    verifyProducerRecords(mockProducer, topic, 0, numElements, keyIsAbsent, 
verifyTimestamp);
+  }
+
+  private static void verifyProducerRecords(
+      MockProducer<Integer, Long> mockProducer,
+      String topic,
+      int startElement,
+      int numElements,
+      boolean keyIsAbsent,
+      boolean verifyTimestamp) {
 
     // verify that appropriate messages are written to kafka
     List<ProducerRecord<Integer, Long>> sent = mockProducer.history();
@@ -1383,7 +1428,7 @@ private static void verifyProducerRecords(
     // sort by values
     sent.sort(Comparator.comparingLong(ProducerRecord::value));
 
-    for (int i = 0; i < numElements; i++) {
+    for (int i = startElement; i < numElements; i++) {
       ProducerRecord<Integer, Long> record = sent.get(i);
       assertEquals(topic, record.topic());
       if (keyIsAbsent) {


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 163718)
    Time Spent: 2h 10m  (was: 2h)

> Add support for dynamic destinations when writing to Kafka
> ----------------------------------------------------------
>
>                 Key: BEAM-5798
>                 URL: https://issues.apache.org/jira/browse/BEAM-5798
>             Project: Beam
>          Issue Type: New Feature
>          Components: io-java-kafka
>            Reporter: Luke Cwik
>            Assignee: Alexey Romanenko
>            Priority: Major
>              Labels: newbie, starter
>          Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Add support for writing to Kafka based upon contents of the data. This is 
> similar to the dynamic destination approach for file IO and other sinks.
>  
> Source of request: 
> https://lists.apache.org/thread.html/a89d1d32ecdb50c42271e805cc01a651ee3623b4df97c39baf4f2053@%3Cuser.beam.apache.org%3E



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to