Skip to content
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

fix RocketMQ Source has no namesrv set error #35

Merged
merged 7 commits into from
Aug 23, 2021
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions .gitignore
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ target/
.DS_Store
!.mvn/wrapper/maven-wrapper.jar
*.versionsBackup
.gradle/

### STS ###
.apt_generated
Expand Down
1 change: 1 addition & 0 deletions build_without_test.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
mvn clean package -Dmaven.test.skip=true
6 changes: 6 additions & 0 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,7 @@
<module>rocketmq-streams-channel-rocketmq</module>
<module>rocketmq-streams-channel-db</module>
<module>rocketmq-streams-channel-http</module>
<module>rocketmq-streams-examples</module>
</modules>

<properties>
Expand Down Expand Up @@ -173,6 +174,11 @@
<artifactId>rocketmq-streams-commons</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-streams-clients</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-streams-configurable</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,6 @@ protected DefaultMQPushConsumer startConsumer() {
});

setOffsetStore(consumer);
addRebalanceCallback(consumer);
consumer.start();

return consumer;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
package org.apache.rocketmq.streams.client.source;

import com.google.common.collect.Sets;

import java.util.Set;

import org.apache.rocketmq.streams.client.transform.DataStream;
import org.apache.rocketmq.streams.common.channel.impl.file.FileSource;
import org.apache.rocketmq.streams.common.channel.source.ISource;
Expand Down Expand Up @@ -49,20 +51,21 @@ public DataStream fromFile(String filePath, Boolean isJsonData) {
return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null);
}

public DataStream fromRocketmq(String topic, String groupName) {
return fromRocketmq(topic, groupName, null, false);
public DataStream fromRocketmq(String topic, String groupName, String namesrvAddress) {
return fromRocketmq(topic, groupName, false, namesrvAddress);
}

public DataStream fromRocketmq(String topic, String groupName, boolean isJson) {
return fromRocketmq(topic, groupName, null, isJson);
public DataStream fromRocketmq(String topic, String groupName, boolean isJson, String namesrvAddress) {
return fromRocketmq(topic, groupName, "*", isJson, namesrvAddress);
}

public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson) {
public DataStream fromRocketmq(String topic, String groupName, String tags, boolean isJson, String namesrvAddress) {
RocketMQSource rocketMQSource = new RocketMQSource();
rocketMQSource.setTopic(topic);
rocketMQSource.setTags(tags);
rocketMQSource.setGroupName(groupName);
rocketMQSource.setJsonData(isJson);
rocketMQSource.setNamesrvAddr(namesrvAddress);
this.mainPipelineBuilder.setSource(rocketMQSource);
return new DataStream(this.mainPipelineBuilder, this.otherPipelineBuilders, null);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,8 @@
package org.apache.rocketmq.streams.client;

import com.alibaba.fastjson.JSONObject;
import netscape.javascript.JSObject;
import org.apache.rocketmq.streams.client.source.DataStreamSource;
import org.apache.rocketmq.streams.client.strategy.CheckpointStrategy;
import org.apache.rocketmq.streams.client.strategy.StateStrategy;
import org.apache.rocketmq.streams.client.strategy.Strategy;
import org.apache.rocketmq.streams.client.transform.window.Time;
import org.apache.rocketmq.streams.client.transform.window.TumblingWindow;
import org.apache.rocketmq.streams.common.functions.MapFunction;
Expand All @@ -45,76 +42,74 @@ public void init() {
@Test
public void testFromFile() {
dataStream
.fromFile("/Users/junjie.cheng/text.txt", false)
.map(message -> message + "--")
.toPrint(1)
.start();
.fromFile("/Users/junjie.cheng/text.txt", false)
.map(message -> message + "--")
.toPrint(1)
.start();
}

@Test
public void testRocketmq() {


DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline");
dataStream
.fromRocketmq("TOPIC_EVENT_SAS_SECURITY_EVENT", "111")
.map(message -> message + "--")
.toPrint(1)
.start();
.fromRocketmq("topic_xxxx01", "consumer_xxxx01", "127.0.0.1:9876")
.map(message -> message + "--")
.toPrint(1)
.start();
}

@Test
public void testDBCheckPoint() {
dataStream
.fromRocketmq("TSG_META_INFO", "")
.map(message -> message + "--")
.toPrint(1)
.with(CheckpointStrategy.db("", "", "", 0L))
.start();
.fromRocketmq("topic_xxxx02", "consumer_xxxx02", "127.0.0.1:9876")
.map(message -> message + "--")
.toPrint(1)
.with(CheckpointStrategy.db("", "", "", 0L))
.start();
}

@Test
public void testFileCheckPoint() {
dataStream
.fromRocketmq("TSG_META_INFO", "")
.map(message -> message + "--")
.toPrint(1)
.with(CheckpointStrategy.mem(0L))
.start();
.fromFile("/Users/junjie.cheng/text.txt", false)
.map(message -> message + "--")
.toPrint(1)
.with(CheckpointStrategy.mem(0L))
.start();
}


@Test
public void testWindow() {
DataStreamSource dataStream = StreamBuilder.dataStream("test_namespace", "graph_pipeline");
dataStream
.fromRocketmq("TSG_META_INFO", "")
.map(new MapFunction<JSONObject, String>() {

@Override
public JSONObject map(String message) throws Exception {
JSONObject msg=JSONObject.parseObject(message);
return msg;
}
})
.window(TumblingWindow.of(Time.seconds(5)))
.groupBy("name","age")
.count("c")
.sum("score","scoreValue")
.toDataSteam()
.toPrint(1)
.with(CheckpointStrategy.db("","","",1000L))
.start();
.fromRocketmq("topic_xxxx03", "consumer_xxxx03", "127.0.0.1:9876")
.map(new MapFunction<JSONObject, String>() {

@Override
public JSONObject map(String message) throws Exception {
JSONObject msg = JSONObject.parseObject(message);
return msg;
}
})
.window(TumblingWindow.of(Time.seconds(5)))
.groupBy("name", "age")
.count("c")
.sum("score", "scoreValue")
.toDataSteam()
.toPrint(1)
.with(CheckpointStrategy.db("", "", "", 1000L))
.start();
}

@Test
public void testBothStrategy() {
dataStream
.fromRocketmq("TSG_META_INFO", "")
.map(message -> message + "--")
.toPrint(1)
.with()
.start();
.fromRocketmq("topic_xxxx04", "consumer_xxxx04", "127.0.0.1:9876")
.map(message -> message + "--")
.toPrint(1)
.with()
.start();
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public void testWindowToMetaq() throws InterruptedException {

protected DataStream createSourceDataStream(){
return StreamBuilder.dataStream("namespace", "name1")
.fromRocketmq(topic,"chris1",true);
.fromRocketmq(topic,"chris1","");
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,16 @@ protected void setJsonObject(JSONObject jsonObject) {
jsonObject.put("source", Base64Utils.encode(InstantiationUtil.serializeObject(source)));
}

@Override
public void removeSplit(Set<String> splitIds) {
source.removeSplit(splitIds);
}

@Override
public void addNewSplit(Set<String> splitIds) {
source.addNewSplit(splitIds);
}

@Override
public Map<String, MessageOffset> getFinishedQueueIdAndOffsets(CheckPointMessage checkPointMessage) {
return sink.getFinishedQueueIdAndOffsets(checkPointMessage);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,4 +74,8 @@ public interface ISource<T extends ISource> extends IConfigurable, IStageBuilder
*/
long getCheckpointTime();

void removeSplit(Set<String> splitIds);

void addNewSplit(Set<String> splitIds);

}
50 changes: 50 additions & 0 deletions rocketmq-streams-examples/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
<?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.
-->
<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">
<parent>
<artifactId>rocketmq-streams</artifactId>
<groupId>org.apache.rocketmq</groupId>
<version>1.0.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>rocketmq-streams-examples</artifactId>
<name>ROCKETMQ STREAMS :: examples</name>
<dependencies>
<dependency>
<groupId>org.apache.rocketmq</groupId>
<artifactId>rocketmq-streams-clients</artifactId>
<exclusions>
<exclusion>
<groupId>ch.qos.logback</groupId>
<artifactId>logback-classic</artifactId>
</exclusion>
</exclusions>
</dependency>
</dependencies>
<packaging>jar</packaging>

<properties>
<file_encoding>UTF-8</file_encoding>
<project.build.sourceEncoding>${file_encoding}</project.build.sourceEncoding>
<maven.compiler.source>8</maven.compiler.source>
<maven.compiler.target>8</maven.compiler.target>
</properties>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.streams.examples.filesource;

import org.apache.rocketmq.streams.client.StreamBuilder;
import org.apache.rocketmq.streams.client.source.DataStreamSource;

public class FileSourceExample {
public static void main(String[] args) {
DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline");
source.fromFile("/your/file/path", false)
.map(message -> message)
.toPrint(1)
.start();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.streams.examples.rocketmqsource;

import org.apache.rocketmq.streams.client.StreamBuilder;
import org.apache.rocketmq.streams.client.source.DataStreamSource;

public class RocketMQSourceExample1 {
public static void main(String[] args) {
DataStreamSource source = StreamBuilder.dataStream("namespace", "pipeline");

source.fromRocketmq(
RocketMQSourceExample2.RMQ_TOPIC,
RocketMQSourceExample2.RMQ_CONSUMER_GROUP_NAME,
RocketMQSourceExample2.NAMESRV_ADDRESS
)
.map(message -> message)
.toPrint(1)
.start();

}
}
Loading