[ 
https://issues.apache.org/jira/browse/ROCKETMQ-82?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16406409#comment-16406409
 ] 

ASF GitHub Bot commented on ROCKETMQ-82:
----------------------------------------

zhouxinyu commented on a change in pull request #45: ROCKETMQ-82: 
RocketMQ-Flink Integration
URL: https://github.com/apache/rocketmq-externals/pull/45#discussion_r175782625
 
 

 ##########
 File path: 
rocketmq-flink/src/main/java/org/apache/rocketmq/flink/RocketMQSink.java
 ##########
 @@ -0,0 +1,189 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 org.apache.rocketmq.flink;
+
+import java.nio.charset.StandardCharsets;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.Validate;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.client.producer.DefaultMQProducer;
+import org.apache.rocketmq.client.producer.SendCallback;
+import org.apache.rocketmq.client.producer.SendResult;
+import org.apache.rocketmq.common.message.Message;
+import org.apache.rocketmq.flink.common.selector.TopicSelector;
+import 
org.apache.rocketmq.flink.common.serialization.KeyValueSerializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The RocketMQSink provides at-least-once reliability guarantees when
+ * checkpoints are enabled and batchFlushOnCheckpoint(true) is set.
+ * Otherwise, the sink reliability guarantees depends on rocketmq producer's 
retry policy.
+ */
+public class RocketMQSink<IN> extends RichSinkFunction<IN> implements 
CheckpointedFunction {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(RocketMQSink.class);
+
+    private transient DefaultMQProducer producer;
+    private boolean async; // false by default
+
+    private Properties props;
+    private TopicSelector<IN> topicSelector;
+    private KeyValueSerializationSchema<IN> serializationSchema;
+
+    private boolean batchFlushOnCheckpoint; // false by default
+    private List<Message> batchList;
+
+    public RocketMQSink(KeyValueSerializationSchema<IN> schema, 
TopicSelector<IN> topicSelector, Properties props) {
+        this.serializationSchema = schema;
+        this.topicSelector = topicSelector;
+        this.props = props;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        Validate.notEmpty(props, "Producer properties can not be empty");
+        Validate.notNull(topicSelector, "TopicSelector can not be null");
+        Validate.notNull(serializationSchema, "KeyValueSerializationSchema can 
not be null");
+
+        producer = new DefaultMQProducer();
+        
producer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        RocketMqConfig.buildProducerConfigs(props, producer);
+
+        batchList = new LinkedList<>();
+
+        if (batchFlushOnCheckpoint && !((StreamingRuntimeContext) 
getRuntimeContext()).isCheckpointingEnabled()) {
+            LOG.warn("Flushing on checkpoint is enabled, but checkpointing is 
not enabled. Disabling flushing.");
+            batchFlushOnCheckpoint = false;
+        }
+
+        try {
+            producer.start();
+        } catch (MQClientException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public void invoke(IN input, Context context) throws Exception {
+        Message msg = prepareMessage(input);
+
+        if (batchFlushOnCheckpoint) {
+            batchList.add(msg);
+            return;
+        }
+
+        if (async) {
+            // async sending
+            try {
+                producer.send(msg, new SendCallback() {
+                    @Override
+                    public void onSuccess(SendResult sendResult) {
+                        LOG.debug("Async send message success!");
+                    }
+
+                    @Override
+                    public void onException(Throwable throwable) {
+                        if (throwable != null) {
+                            LOG.error("Async send message failure!", 
throwable);
+                        }
+                    }
+                });
+            } catch (Exception e) {
+                LOG.error("Async send message failure!", e);
+            }
+        } else {
+            // sync sending, will return a SendResult
+            try {
+                SendResult result = producer.send(msg);
+                if (LOG.isDebugEnabled()) {
 
 Review comment:
   No need to check whether debug is enabled.

----------------------------------------------------------------
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


> Add the RocketMQ plugin for the Apache Flink
> --------------------------------------------
>
>                 Key: ROCKETMQ-82
>                 URL: https://issues.apache.org/jira/browse/ROCKETMQ-82
>             Project: Apache RocketMQ
>          Issue Type: Wish
>          Components: rocketmq-externals
>            Reporter: vongosling
>            Assignee: Xin Wang
>            Priority: Minor
>
> A few basic data sources and sinks are built into Flink and are always 
> available. The [predefined data 
> sources|https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/datastream_api.html#data-sources]
>  include reading from files, directories, and sockets, and ingesting data 
> from collections and iterators. The [predefined data 
> sinks|https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/datastream_api.html#data-sinks]
>  support writing to files, to stdout and stderr, and to sockets.
> The connector is the integration plugin for RocketMQ and Flink, some details 
> in here,
> 1. 
> [https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/connectors/index.html]
> 2. [https://github.com/apache/rocketmq-externals/tree/master/rocketmq-flink]



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

Reply via email to