-
Notifications
You must be signed in to change notification settings - Fork 1.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
IGNITE-4539: RocketMQ Streamer implementation. #1571
Closed
Closed
Changes from 7 commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
b5c9318
IGNITE-4539: RocketMQ data streamer.
shroman dc2eed6
Changed RocketMQ version to the latest release.
shroman b8a69ab
IGNITE-4539: reworked RocketMQ startup.
shroman dadf826
Merger with master.
shroman 0adfd27
IGNITE-4539: Reworked topic creation.
shroman 77228ab
Merged with master.
shroman 697bc5e
Updated RocketMQ dependencies.
shroman bdf1820
Broker scope to test.
shroman File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,25 @@ | ||
Apache Ignite RocketMQ Streamer Module | ||
-------------------------------------- | ||
|
||
Apache Ignite RocketMQ Streamer module provides streaming from RocketMQ to Ignite cache. | ||
|
||
To use Ignite RocketMQ Streamer module, first import it to your Maven project. | ||
|
||
<project xmlns="http://maven.apache.org/POM/4.0.0" | ||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 | ||
http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
... | ||
<dependencies> | ||
... | ||
<dependency> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-rocketmq</artifactId> | ||
<version>${ignite.version}</version> | ||
</dependency> | ||
... | ||
</dependencies> | ||
... | ||
</project> | ||
|
||
Then, initialize and start it as, for instance, done in RocketMQStreamerTest.java. |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,80 @@ | ||
<?xml version="1.0" encoding="UTF-8"?> | ||
|
||
<!-- | ||
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. | ||
--> | ||
|
||
<!-- | ||
POM file. | ||
--> | ||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" | ||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> | ||
<modelVersion>4.0.0</modelVersion> | ||
|
||
<parent> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-parent</artifactId> | ||
<version>1</version> | ||
<relativePath>../../parent</relativePath> | ||
</parent> | ||
|
||
<artifactId>ignite-rocketmq</artifactId> | ||
<version>2.0.0-SNAPSHOT</version> | ||
<url>http://ignite.apache.org</url> | ||
|
||
<dependencies> | ||
<dependency> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-core</artifactId> | ||
<version>${project.version}</version> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.rocketmq</groupId> | ||
<artifactId>rocketmq-broker</artifactId> | ||
<version>${rocketmq.version}</version> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.rocketmq</groupId> | ||
<artifactId>rocketmq-namesrv</artifactId> | ||
<version>${rocketmq.version}</version> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-spring</artifactId> | ||
<version>${project.version}</version> | ||
<scope>test</scope> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-core</artifactId> | ||
<version>${project.version}</version> | ||
<type>test-jar</type> | ||
<scope>test</scope> | ||
</dependency> | ||
|
||
<dependency> | ||
<groupId>org.apache.ignite</groupId> | ||
<artifactId>ignite-log4j</artifactId> | ||
<version>${project.version}</version> | ||
<scope>test</scope> | ||
</dependency> | ||
</dependencies> | ||
|
||
</project> |
151 changes: 151 additions & 0 deletions
151
modules/rocketmq/src/main/java/org/apache/ignite/stream/rocketmq/RocketMQStreamer.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,151 @@ | ||
/* | ||
* 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.ignite.stream.rocketmq; | ||
|
||
import java.util.List; | ||
import org.apache.ignite.IgniteDataStreamer; | ||
import org.apache.ignite.IgniteException; | ||
import org.apache.ignite.IgniteLogger; | ||
import org.apache.ignite.internal.util.typedef.internal.A; | ||
import org.apache.ignite.stream.StreamAdapter; | ||
import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; | ||
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyContext; | ||
import org.apache.rocketmq.client.consumer.listener.ConsumeConcurrentlyStatus; | ||
import org.apache.rocketmq.client.consumer.listener.MessageListenerConcurrently; | ||
import org.apache.rocketmq.client.exception.MQClientException; | ||
import org.apache.rocketmq.common.consumer.ConsumeFromWhere; | ||
import org.apache.rocketmq.common.message.MessageExt; | ||
|
||
/** | ||
* Streamer that subscribes to a RocketMQ topic amd feeds messages into {@link IgniteDataStreamer} instance. | ||
*/ | ||
public class RocketMQStreamer<K, V> extends StreamAdapter<List<MessageExt>, K, V> implements MessageListenerConcurrently { | ||
/** Logger. */ | ||
private IgniteLogger log; | ||
|
||
/** RocketMQ consumer. */ | ||
private DefaultMQPushConsumer consumer; | ||
|
||
/** State. */ | ||
private volatile boolean stopped = true; | ||
|
||
/** Topic to subscribe to. */ | ||
private String topic; | ||
|
||
/** Consumer group. */ | ||
private String consumerGrp; | ||
|
||
/** Name server address. */ | ||
private String nameSrvAddr; | ||
|
||
/** | ||
* Starts streamer. | ||
* | ||
* @throws IgniteException If failed. | ||
*/ | ||
public void start() { | ||
if (!stopped) | ||
throw new IgniteException("Attempted to start an already started RocketMQ streamer"); | ||
|
||
// validate parameters. | ||
A.notNull(getStreamer(), "streamer"); | ||
A.notNull(getIgnite(), "ignite"); | ||
A.notNull(topic, "topic"); | ||
A.notNull(consumerGrp, "consumer group"); | ||
A.notNullOrEmpty(nameSrvAddr, "nameserver address"); | ||
A.ensure(null != getMultipleTupleExtractor(), "Multiple tuple extractor must be configured"); | ||
|
||
log = getIgnite().log(); | ||
|
||
consumer = new DefaultMQPushConsumer(consumerGrp); | ||
|
||
consumer.setNamesrvAddr(nameSrvAddr); | ||
consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_LAST_OFFSET); | ||
|
||
try { | ||
consumer.subscribe(topic, "*"); | ||
} | ||
catch (MQClientException e) { | ||
throw new IgniteException("Failed to subscribe to " + topic, e); | ||
} | ||
|
||
consumer.registerMessageListener(this); | ||
|
||
try { | ||
consumer.start(); | ||
} | ||
catch (MQClientException e) { | ||
throw new IgniteException("Failed to start the streamer", e); | ||
} | ||
|
||
stopped = false; | ||
} | ||
|
||
/** | ||
* Stops streamer. | ||
*/ | ||
public void stop() { | ||
if (consumer != null) | ||
consumer.shutdown(); | ||
|
||
stopped = true; | ||
} | ||
|
||
/** | ||
* Implements {@link MessageListenerConcurrently#consumeMessage(List, ConsumeConcurrentlyContext)} to receive | ||
* messages. | ||
* | ||
* {@inheritDoc} | ||
*/ | ||
@Override public ConsumeConcurrentlyStatus consumeMessage(List<MessageExt> msgs, | ||
ConsumeConcurrentlyContext context) { | ||
if (log.isDebugEnabled()) | ||
log.debug("Received " + msgs.size() + " messages"); | ||
|
||
addMessage(msgs); | ||
|
||
return ConsumeConcurrentlyStatus.CONSUME_SUCCESS; | ||
} | ||
|
||
/** | ||
* Sets the topic to subscribe to. | ||
* | ||
* @param topic The topic to subscribe to. | ||
*/ | ||
public void setTopic(String topic) { | ||
this.topic = topic; | ||
} | ||
|
||
/** | ||
* Sets the name of the consumer group. | ||
* | ||
* @param consumerGrp Consumer group name. | ||
*/ | ||
public void setConsumerGrp(String consumerGrp) { | ||
this.consumerGrp = consumerGrp; | ||
} | ||
|
||
/** | ||
* Sets the name server address. | ||
* | ||
* @param nameSrvAddr Name server address | ||
*/ | ||
public void setNameSrvAddr(String nameSrvAddr) { | ||
this.nameSrvAddr = nameSrvAddr; | ||
} | ||
} |
21 changes: 21 additions & 0 deletions
21
modules/rocketmq/src/main/java/org/apache/ignite/stream/rocketmq/package-info.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,21 @@ | ||
/* | ||
* 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. | ||
*/ | ||
|
||
/** | ||
* Contains implementation of RocketMQStreamer tests. | ||
*/ | ||
package org.apache.ignite.stream.rocketmq; |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have checked your broker dependency is just for test. Could you scope this dependency to test ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks! Modified.