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

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

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

 ##########
 File path: 
rocketmq-flink/src/main/java/org/apache/rocketmq/flink/RocketMQSource.java
 ##########
 @@ -0,0 +1,324 @@
+/**
+ * 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.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.commons.lang.Validate;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+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.source.RichParallelSourceFunction;
+import org.apache.rocketmq.client.consumer.DefaultMQPullConsumer;
+import org.apache.rocketmq.client.consumer.MQPullConsumerScheduleService;
+import org.apache.rocketmq.client.consumer.PullResult;
+import org.apache.rocketmq.client.consumer.PullTaskCallback;
+import org.apache.rocketmq.client.consumer.PullTaskContext;
+import org.apache.rocketmq.client.exception.MQClientException;
+import org.apache.rocketmq.common.message.MessageExt;
+import org.apache.rocketmq.common.message.MessageQueue;
+import 
org.apache.rocketmq.flink.common.serialization.KeyValueDeserializationSchema;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static 
org.apache.rocketmq.flink.RocketMqConfig.CONSUMER_OFFSET_EARLIEST;
+import static org.apache.rocketmq.flink.RocketMqConfig.CONSUMER_OFFSET_LATEST;
+import static 
org.apache.rocketmq.flink.RocketMqConfig.CONSUMER_OFFSET_TIMESTAMP;
+import static org.apache.rocketmq.flink.RocketMqUtils.getInteger;
+import static org.apache.rocketmq.flink.RocketMqUtils.getLong;
+
+/**
+ * The RocketMQSource is based on RocketMQ pull consumer mode,
+ * and provides exactly once reliability guarantees when checkpoints are 
enabled.
+ * Otherwise, the source doesn't provide any reliability guarantees.
+ */
+public class RocketMQSource<OUT> extends RichParallelSourceFunction<OUT>
+    implements CheckpointedFunction, ResultTypeQueryable<OUT> {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(RocketMQSource.class);
+
+    private transient MQPullConsumerScheduleService 
pullConsumerScheduleService;
+    private DefaultMQPullConsumer consumer;
+
+    private KeyValueDeserializationSchema<OUT> schema;
+
+    private RunningChecker runningChecker;
+
+    private transient ListState<Tuple2<MessageQueue, Long>> unionOffsetStates;
+    private Map<MessageQueue, Long> offsetTable;
+    private Map<MessageQueue, Long> restoredOffsets;
+
+    private Properties props;
+    private String topic;
+    private String group;
+
+    private static final String OFFSETS_STATE_NAME = 
"topic-partition-offset-states";
+
+    private transient volatile boolean restored;
+
+    public RocketMQSource(KeyValueDeserializationSchema<OUT> schema, 
Properties props) {
+        this.schema = schema;
+        this.props = props;
+    }
+
+    @Override
+    public void open(Configuration parameters) throws Exception {
+        LOG.debug("source open....");
+        Validate.notEmpty(props, "Consumer properties can not be empty");
+        Validate.notNull(schema, "KeyValueDeserializationSchema can not be 
null");
+
+        this.topic = props.getProperty(RocketMqConfig.CONSUMER_TOPIC);
+        this.group = props.getProperty(RocketMqConfig.CONSUMER_GROUP);
+
+        Validate.notEmpty(topic, "Consumer topic can not be empty");
+        Validate.notEmpty(group, "Consumer group can not be empty");
+
+        if (offsetTable == null) {
+            offsetTable = new ConcurrentHashMap<>();
+        }
+        if (restoredOffsets == null) {
+            restoredOffsets = new ConcurrentHashMap<>();
+        }
+
+        runningChecker = new RunningChecker();
+
+        pullConsumerScheduleService = new MQPullConsumerScheduleService(group);
+        consumer = pullConsumerScheduleService.getDefaultMQPullConsumer();
+
+        
consumer.setInstanceName(String.valueOf(getRuntimeContext().getIndexOfThisSubtask()));
+        RocketMqConfig.buildConsumerConfigs(props, consumer);
+    }
+
+    @Override
+    public void run(SourceContext context) throws Exception {
+        LOG.debug("source run....");
+        // The lock that guarantees that record emission and state updates are 
atomic,
+        // from the view of taking a checkpoint.
+        final Object lock = context.getCheckpointLock();
+
+        int delayWhenMessageNotFound = getInteger(props, 
RocketMqConfig.CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND,
+            RocketMqConfig.DEFAULT_CONSUMER_DELAY_WHEN_MESSAGE_NOT_FOUND);
+
+        String tag = props.getProperty(RocketMqConfig.CONSUMER_TAG, 
RocketMqConfig.DEFAULT_CONSUMER_TAG);
+
+        int pullPoolSize = getInteger(props, 
RocketMqConfig.CONSUMER_PULL_POOL_SIZE,
+            RocketMqConfig.DEFAULT_CONSUMER_PULL_POOL_SIZE);
+
+        int pullBatchSize = getInteger(props, 
RocketMqConfig.CONSUMER_BATCH_SIZE,
+            RocketMqConfig.DEFAULT_CONSUMER_BATCH_SIZE);
+
+        pullConsumerScheduleService.setPullThreadNums(pullPoolSize);
+        pullConsumerScheduleService.registerPullTaskCallback(topic, new 
PullTaskCallback() {
+
+            @Override
+            public void doPullTask(MessageQueue mq, PullTaskContext 
pullTaskContext) {
+                try {
+                    long offset = getMessageQueueOffset(mq);
+                    if (offset < 0) {
+                        return;
+                    }
+
+                    PullResult pullResult = consumer.pull(mq, tag, offset, 
pullBatchSize);
+                    switch (pullResult.getPullStatus()) {
+                        case FOUND:
+                            List<MessageExt> messages = 
pullResult.getMsgFoundList();
+                            for (MessageExt msg : messages) {
+                                byte[] key = msg.getKeys() != null ? 
msg.getKeys().getBytes(StandardCharsets.UTF_8) : null;
+                                byte[] value = msg.getBody();
+                                OUT data = schema.deserializeKeyAndValue(key, 
value);
+
+                                // output and state update are atomic
+                                synchronized (lock) {
+                                    context.collectWithTimestamp(data, 
msg.getBornTimestamp());
+                                    putMessageQueueOffset(mq, 
pullResult.getNextBeginOffset());
 
 Review comment:
   Yes. We need the `nextBeginOffset` in the `PullResult` to update the offset 
table which will be checkpointing when `snapshotState` invoked .

----------------------------------------------------------------
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:
[email protected]


> 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