pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] sijie commented on a change in pull request #1708: Refactor functions to use Sink interface
Date Wed, 02 May 2018 07:20:38 GMT
sijie commented on a change in pull request #1708: Refactor functions to use Sink interface
URL: https://github.com/apache/incubator-pulsar/pull/1708#discussion_r185410486
 
 

 ##########
 File path: pulsar-functions/instance/src/main/java/org/apache/pulsar/functions/sink/PulsarSink.java
 ##########
 @@ -18,5 +18,249 @@
  */
 package org.apache.pulsar.functions.sink;
 
-public class PulsarSink {
+import com.google.common.annotations.VisibleForTesting;
+import lombok.AccessLevel;
+import lombok.Getter;
+import lombok.extern.slf4j.Slf4j;
+import net.jodah.typetools.TypeResolver;
+import org.apache.pulsar.client.api.Consumer;
+import org.apache.pulsar.client.api.ConsumerEventListener;
+import org.apache.pulsar.client.api.Message;
+import org.apache.pulsar.client.api.MessageBuilder;
+import org.apache.pulsar.client.api.Producer;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.connect.core.RecordContext;
+import org.apache.pulsar.functions.api.SerDe;
+import org.apache.pulsar.functions.api.utils.DefaultSerDe;
+import org.apache.pulsar.functions.instance.InstanceUtils;
+import org.apache.pulsar.functions.instance.producers.AbstractOneOuputTopicProducers;
+import org.apache.pulsar.functions.instance.producers.MultiConsumersOneOuputTopicProducers;
+import org.apache.pulsar.functions.instance.producers.Producers;
+import org.apache.pulsar.functions.source.PulsarRecord;
+import org.apache.pulsar.functions.utils.FunctionConfig;
+
+import java.util.Base64;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+@Slf4j
+public class PulsarSink<T> implements RuntimeSink<T> {
+
+    private PulsarClient client;
+    private PulsarSinkConfig pulsarSinkConfig;
+    private SerDe<T> outputSerDe;
+
+    private PulsarSinkProcessor pulsarSinkProcessor;
+
+    private interface PulsarSinkProcessor {
+        void initializeOutputProducer(String outputTopic) throws Exception;
+
+        void sendOutputMessage(MessageBuilder outputMsgBuilder,
+                               PulsarRecord pulsarRecord) throws Exception;
+
+        void close() throws Exception;
+    }
+
+    private class PulsarSinkAtMostOnceProcessor implements PulsarSinkProcessor {
+        private Producer<byte[]> producer;
+
+        @Override
+        public void initializeOutputProducer(String outputTopic) throws Exception {
+            this.producer = AbstractOneOuputTopicProducers.createProducer(
+                    client, pulsarSinkConfig.getTopic());
+        }
+
+        @Override
+        public void sendOutputMessage(MessageBuilder outputMsgBuilder,
+                                      PulsarRecord pulsarRecord) throws Exception {
+            Message<byte[]> outputMsg = outputMsgBuilder.build();
+            this.producer.sendAsync(outputMsg);
+        }
+
+        @Override
+        public void close() throws Exception {
+            if (null != producer) {
+                try {
+                    producer.close();
+                } catch (PulsarClientException e) {
+                    log.warn("Fail to close producer for processor {}", pulsarSinkConfig.getTopic(),
e);
+                }
+            }
+        }
+    }
+
+    private class PulsarSinkAtLeastOnceProcessor implements PulsarSinkProcessor {
+        private Producer<byte[]> producer;
+
+        @Override
+        public void initializeOutputProducer(String outputTopic) throws Exception {
+            this.producer = AbstractOneOuputTopicProducers.createProducer(
+                    client, pulsarSinkConfig.getTopic());
+        }
+
+        @Override
+        public void sendOutputMessage(MessageBuilder outputMsgBuilder,
+                                      PulsarRecord pulsarRecord) throws Exception {
+            Message<byte[]> outputMsg = outputMsgBuilder.build();
+            this.producer.sendAsync(outputMsg).thenAccept(messageId -> pulsarRecord.ack());
+        }
+
+        @Override
+        public void close() throws Exception {
+            if (null != producer) {
+                try {
+                    producer.close();
+                } catch (PulsarClientException e) {
+                    log.warn("Fail to close producer for processor {}", pulsarSinkConfig.getTopic(),
e);
+                }
+            }
+        }
+    }
+
+    private class PulsarSinkEffectivelyOnceProcessor implements PulsarSinkProcessor, ConsumerEventListener
{
+
+        @Getter(AccessLevel.PACKAGE)
+        protected Producers outputProducer;
+
+        @Override
+        public void initializeOutputProducer(String outputTopic) throws Exception {
+            outputProducer = new MultiConsumersOneOuputTopicProducers(client, outputTopic);
+            outputProducer.initialize();
+        }
+
+        @Override
+        public void sendOutputMessage(MessageBuilder outputMsgBuilder, PulsarRecord pulsarRecord)
+                throws Exception {
+
+            // assign sequence id to output message for idempotent producing
+            outputMsgBuilder = outputMsgBuilder
+                    .setSequenceId(pulsarRecord.getRecordSequence());
+
+            // currently on PulsarRecord
+            Producer producer = outputProducer.getProducer(pulsarRecord.getTopicName(),
+                    Integer.parseInt(pulsarRecord.getPartitionId()));
+
+            org.apache.pulsar.client.api.Message outputMsg = outputMsgBuilder.build();
+            producer.sendAsync(outputMsg)
+                    .thenAccept(messageId -> pulsarRecord.ack())
+                    .join();
+        }
+
+        @Override
+        public void close() throws Exception {
+            // kill the result producer
+            if (null != outputProducer) {
+                outputProducer.close();
+                outputProducer = null;
+            }
+        }
+
+        @Override
+        public void becameActive(Consumer<?> consumer, int partitionId) {
+            // if the instance becomes active for a given topic partition,
+            // open a producer for the results computed from this topic partition.
+            if (null != outputProducer) {
+                try {
+                    this.outputProducer.getProducer(consumer.getTopic(), partitionId);
+                } catch (PulsarClientException e) {
+                    // this can be ignored, because producer can be lazily created when accessing
it.
+                    log.warn("Fail to create a producer for results computed from messages
of topic: {}, partition: {}",
+                            consumer.getTopic(), partitionId);
+                }
+            }
+        }
+
+        @Override
+        public void becameInactive(Consumer<?> consumer, int partitionId) {
+            if (null != outputProducer) {
+                // if I lost the ownership of a partition, close its corresponding topic
partition.
+                // this is to allow the new active consumer be able to produce to the result
topic.
+                this.outputProducer.closeProducer(consumer.getTopic(), partitionId);
+            }
+        }
+    }
+
+    public PulsarSink(PulsarClient client, PulsarSinkConfig pulsarSinkConfig) {
+        this.client = client;
+        this.pulsarSinkConfig = pulsarSinkConfig;
+    }
+
+    @Override
+    public void open(Map<String, Object> config) throws Exception {
+
+        // Setup Serialization/Deserialization
+        setupSerDe();
+
+        FunctionConfig.ProcessingGuarantees processingGuarantees = this.pulsarSinkConfig.getProcessingGuarantees();
+        switch (processingGuarantees) {
+            case ATMOST_ONCE:
+                this.pulsarSinkProcessor = new PulsarSinkAtMostOnceProcessor();
+                break;
+            case ATLEAST_ONCE:
+                this.pulsarSinkProcessor = new PulsarSinkAtLeastOnceProcessor();
+                break;
+            case EFFECTIVELY_ONCE:
+                this.pulsarSinkProcessor = new PulsarSinkEffectivelyOnceProcessor();
+                break;
+        }
+        this.pulsarSinkProcessor.initializeOutputProducer(this.pulsarSinkConfig.getTopic());
+    }
+
+    @Override
+    public CompletableFuture<Void> write(T value) throws Exception {
+        return null;
 
 Review comment:
   `throw new UnsupportedException(..)`?

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services

Mime
View raw message